You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by adeneche <gi...@git.apache.org> on 2016/05/15 16:21:20 UTC

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

GitHub user adeneche opened a pull request:

    https://github.com/apache/drill/pull/503

    DRILL-4676: Foreman.moveToState can block forever if called by the fo\u2026

    \u2026reman thread while the query is still being setup

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/adeneche/incubator-drill DRILL-4676

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/503.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #503
    
----
commit 8608613e8e1f3e847ffde5b233442a0e78f1d1cc
Author: adeneche <ad...@gmail.com>
Date:   2016-05-12T15:32:52Z

    DRILL-4676: Foreman.moveToState can block forever if called by the foreman thread while the query is still being setup

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444973
  
    --- Diff: common/src/main/java/org/apache/drill/common/EventProcessor.java ---
    @@ -57,25 +58,35 @@ public EventProcessor() {
        */
       public void sendEvent(final T newEvent) {
         synchronized (queuedEvents) {
    -      if (isProcessing) {
    -        queuedEvents.addLast(newEvent);
    +      queuedEvents.addLast(newEvent);
    +      if (!started || isProcessing) {
    --- End diff --
    
    Update the method docs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444982
  
    --- Diff: common/src/main/java/org/apache/drill/common/EventProcessor.java ---
    @@ -57,25 +58,35 @@ public EventProcessor() {
        */
       public void sendEvent(final T newEvent) {
         synchronized (queuedEvents) {
    -      if (isProcessing) {
    -        queuedEvents.addLast(newEvent);
    +      queuedEvents.addLast(newEvent);
    +      if (!started || isProcessing) {
             return;
           }
     
           isProcessing = true;
         }
     
    +    processEvents();
    +  }
    +
    +  public void start() {
    +    if (started) {
    +      return;
    +    }
    +
    +    synchronized (queuedEvents) {
    +      started = true;
    +      isProcessing = true;
    +    }
    +
    +    processEvents();
    +  }
    +
    +  private void processEvents() {
         @SuppressWarnings("resource")
         final DeferredException deferredException = new DeferredException();
    -    T event = newEvent;
    +    T event;
    --- End diff --
    
    Move this declaration inside the loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by hnfgns <gi...@git.apache.org>.
Github user hnfgns commented on the pull request:

    https://github.com/apache/drill/pull/503#issuecomment-219573757
  
    I see that you are enabling task hand-off for the first task in the serializer as well. I am not convinced that this approach will for sure fix the deadlock case for RPCs and foreman failure case. I do not have time now but I will think about this issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63431381
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    We were assuming that the Foreman thread would never call Foreman.StateListener.moveToState() and it would be called by another (rpc) thread instead.
    It turns out when the foreman is submitting remote fragments, RpcBus.send() could actually cause the foreman thread to call moveToState directly


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63578150
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -163,7 +161,7 @@ public Foreman(final WorkerBee bee, final DrillbitContext drillbitContext,
     
         queryContext = new QueryContext(connection.getSession(), drillbitContext, queryId);
         queryManager = new QueryManager(queryId, queryRequest, drillbitContext.getStoreProvider(),
    -        drillbitContext.getClusterCoordinator(), stateListener, this); // TODO reference escapes before ctor is complete via stateListener, this
    +        drillbitContext.getClusterCoordinator(), this); // TODO reference escapes before ctor is complete via stateListener, this
    --- End diff --
    
    removed comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by jacques-n <gi...@git.apache.org>.
Github user jacques-n commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63294968
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -115,6 +115,8 @@
       private static final ObjectMapper MAPPER = new ObjectMapper();
       private static final long RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
     
    +  private Thread myThreadRef; // Thread that is currently executing the Foreman.
    --- End diff --
    
    I think you need either volatile or an atomic reference here since moveToState can be executed by a thread other than the one running the fragment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63302974
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -115,6 +115,8 @@
       private static final ObjectMapper MAPPER = new ObjectMapper();
       private static final long RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
     
    +  private Thread myThreadRef; // Thread that is currently executing the Foreman.
    --- End diff --
    
    Is it really necessary ? myThreadRef is used to force the foreman thread to throw an exception when calling moveToState. Any other thread will ignore that (no matter what value myThreadRef has) and block on the latch instead. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63578473
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1191,7 +1195,7 @@ public void failed(final RpcException ex) {
           } else {
             // since this won't be waited on, we can wait to deliver this event once the Foreman is ready
             logger.debug("Failure while sending fragment.  Stopping query.", ex);
    -        stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    makes sense


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by hnfgns <gi...@git.apache.org>.
Github user hnfgns commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63305469
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -115,6 +115,8 @@
       private static final ObjectMapper MAPPER = new ObjectMapper();
       private static final long RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
     
    +  private Thread myThreadRef; // Thread that is currently executing the Foreman.
    --- End diff --
    
    I believe volatile is unneeded here. In my opinion keeping myThreadRef in heap is also unneeded or seems wrong even. I will detail down.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63573819
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -163,7 +161,7 @@ public Foreman(final WorkerBee bee, final DrillbitContext drillbitContext,
     
         queryContext = new QueryContext(connection.getSession(), drillbitContext, queryId);
         queryManager = new QueryManager(queryId, queryRequest, drillbitContext.getStoreProvider(),
    -        drillbitContext.getClusterCoordinator(), stateListener, this); // TODO reference escapes before ctor is complete via stateListener, this
    +        drillbitContext.getClusterCoordinator(), this); // TODO reference escapes before ctor is complete via stateListener, this
    --- End diff --
    
    Update comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63576357
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1191,7 +1195,7 @@ public void failed(final RpcException ex) {
           } else {
             // since this won't be waited on, we can wait to deliver this event once the Foreman is ready
             logger.debug("Failure while sending fragment.  Stopping query.", ex);
    -        stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    what do you mean by "deferring this" ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63442039
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -296,7 +295,7 @@ public void run() {
            * would wait on the cancelling thread to signal a resume and the cancelling thread would wait on the Foreman
            * to accept events.
            */
    -      acceptExternalEvents.countDown();
    +      stateSwitch.start();
    --- End diff --
    
    This defers all state transitions until the thread is done. This changes the behavior when queuing is enabled, the user will not know if the query has moved from ENQUEUED to STARTING.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444968
  
    --- Diff: common/src/main/java/org/apache/drill/common/EventProcessor.java ---
    @@ -37,6 +37,7 @@
     public abstract class EventProcessor<T> {
       private final LinkedList<T> queuedEvents = new LinkedList<>();
       private volatile boolean isProcessing = false;
    +  private volatile boolean started;
    --- End diff --
    
    Explicitly set to false as above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63577164
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -869,44 +861,44 @@ protected void processEvent(final StateEvent event) {
              * cause this to be called recursively.
              */
             switch (newState) {
    -        case CANCELLATION_REQUESTED: {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444977
  
    --- Diff: common/src/main/java/org/apache/drill/common/EventProcessor.java ---
    @@ -57,25 +58,35 @@ public EventProcessor() {
        */
       public void sendEvent(final T newEvent) {
         synchronized (queuedEvents) {
    -      if (isProcessing) {
    -        queuedEvents.addLast(newEvent);
    +      queuedEvents.addLast(newEvent);
    +      if (!started || isProcessing) {
             return;
           }
     
           isProcessing = true;
         }
     
    +    processEvents();
    +  }
    +
    +  public void start() {
    --- End diff --
    
    Please add docs here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444985
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,8 +1220,6 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    --- End diff --
    
    With this change, compared to Foreman#moveToState, this listener provides no additional functionality. Please remove this class, and update the docs for Foreman#moveToState accordingly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63430234
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    This won't work when submitting remote fragments as the moveToState method can be called either by the foreman thread or by an rpc thread.
    
    Anyway, I'm working on a better fix that would remove the latch completely.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63444805
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -296,7 +295,7 @@ public void run() {
            * would wait on the cancelling thread to signal a resume and the cancelling thread would wait on the Foreman
            * to accept events.
            */
    -      acceptExternalEvents.countDown();
    +      stateSwitch.start();
    --- End diff --
    
    that is true indeed, I didn't notice the Foreman was also using the event queue before it counted down the latch. I will update the fix to respect this behavior


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63573488
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -869,44 +861,44 @@ protected void processEvent(final StateEvent event) {
              * cause this to be called recursively.
              */
             switch (newState) {
    -        case CANCELLATION_REQUESTED: {
    --- End diff --
    
    Can you undo spacing changes? `switch` and `case` should be at the same indentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on the pull request:

    https://github.com/apache/drill/pull/503#issuecomment-219583987
  
    I made a quick change to address Sudheesh concern about deferring processing events when the query is queued. This is a WIP to see if I am not missing anything obvious, will clean/rename/document once we agree this is a correct fix


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63577241
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1191,7 +1195,7 @@ public void failed(final RpcException ex) {
           } else {
             // since this won't be waited on, we can wait to deliver this event once the Foreman is ready
             logger.debug("Failure while sending fragment.  Stopping query.", ex);
    -        stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    I just noticed the comment above, which captures what I meant: we can wait to deliver this event once the Foreman is ready.
    
    Maybe add: this case is only for leaf fragments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63573505
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1206,42 +1210,18 @@ public void interrupted(final InterruptedException e) {
       }
     
       /**
    -   * Provides gated access to state transitions.
    -   *
    -   * <p>The StateListener waits on a latch before delivery state transitions to the Foreman. The
    -   * latch will be tripped when the Foreman is sufficiently set up that it can receive and process
    -   * external events from other threads.
    -   */
    -  public class StateListener {
    -    /**
    -     * Move the Foreman to the specified new state.
    -     *
    -     * @param newState the state to move to
    -     * @param ex if moving to a failure state, the exception that led to the failure; used for reporting
    -     *   to the user
    -     */
    -    public void moveToState(final QueryState newState, final Exception ex) {
    -      acceptExternalEvents.awaitUninterruptibly();
    -
    -      Foreman.this.moveToState(newState, ex);
    -    }
    -  }
    -
    -  /**
        * Listens for the status of the RPC response sent to the user for the query.
        */
       private class ResponseSendListener extends BaseRpcOutcomeListener<Ack> {
         @Override
         public void failed(final RpcException ex) {
           logger.info("Failure while trying communicate query result to initiating client. " +
                   "This would happen if a client is disconnected before response notice can be sent.", ex);
    -      stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    Please add a note about this in the commit message.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/503


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63574135
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1191,7 +1195,7 @@ public void failed(final RpcException ex) {
           } else {
             // since this won't be waited on, we can wait to deliver this event once the Foreman is ready
             logger.debug("Failure while sending fragment.  Stopping query.", ex);
    -        stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    Please add a note about deferring this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63577142
  
    --- Diff: common/src/main/java/org/apache/drill/common/EventProcessor.java ---
    @@ -57,25 +58,35 @@ public EventProcessor() {
        */
       public void sendEvent(final T newEvent) {
         synchronized (queuedEvents) {
    -      if (isProcessing) {
    -        queuedEvents.addLast(newEvent);
    +      queuedEvents.addLast(newEvent);
    +      if (!started || isProcessing) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on the pull request:

    https://github.com/apache/drill/pull/503#issuecomment-219794472
  
    addressed Sudheesh comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63578261
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1206,42 +1210,18 @@ public void interrupted(final InterruptedException e) {
       }
     
       /**
    -   * Provides gated access to state transitions.
    -   *
    -   * <p>The StateListener waits on a latch before delivery state transitions to the Foreman. The
    -   * latch will be tripped when the Foreman is sufficiently set up that it can receive and process
    -   * external events from other threads.
    -   */
    -  public class StateListener {
    -    /**
    -     * Move the Foreman to the specified new state.
    -     *
    -     * @param newState the state to move to
    -     * @param ex if moving to a failure state, the exception that led to the failure; used for reporting
    -     *   to the user
    -     */
    -    public void moveToState(final QueryState newState, final Exception ex) {
    -      acceptExternalEvents.awaitUninterruptibly();
    -
    -      Foreman.this.moveToState(newState, ex);
    -    }
    -  }
    -
    -  /**
        * Listens for the status of the RPC response sent to the user for the query.
        */
       private class ResponseSendListener extends BaseRpcOutcomeListener<Ack> {
         @Override
         public void failed(final RpcException ex) {
           logger.info("Failure while trying communicate query result to initiating client. " +
                   "This would happen if a client is disconnected before response notice can be sent.", ex);
    -      stateListener.moveToState(QueryState.FAILED, ex);
    --- End diff --
    
    will add a note in the squashed commit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63308753
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    In practice, the foreman thread is never meant to call moveToState otherwise it would block forever on the latch. The only cases I've seen this happen are when the foreman [fails to submit a remote fragment](https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java#L1194), and when the foreman [fails to send the final state to the user](https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java#L1238)
    In both cases, the default behavior is to have another thread (rpc) call moveToState, but sometimes, [RpcBus.send](https://github.com/apache/drill/blob/master/exec/rpc/src/main/java/org/apache/drill/exec/rpc/RpcBus.java#L126) causes the foreman thread to explicitly call moveToState.
    
    For example, in the second case, throwing an exception in moveToState will cause the foreman to stop submitting the remaining remote fragments and it will be propagated to Foreman.run where it will cause the foreman to [move to a failed state](https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java#L264)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by hnfgns <gi...@git.apache.org>.
Github user hnfgns commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63305602
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    So now that we are throwing an exception on foreman thread how will we move to failed state in case foreman fails?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by hnfgns <gi...@git.apache.org>.
Github user hnfgns commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63430594
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    Let me ask you this: what was the primary cause of deadlock came out of your investigation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on the pull request:

    https://github.com/apache/drill/pull/503#issuecomment-219803152
  
    +1 (some minor comments)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by hnfgns <gi...@git.apache.org>.
Github user hnfgns commented on a diff in the pull request:

    https://github.com/apache/drill/pull/503#discussion_r63428428
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -1221,6 +1223,12 @@ public void interrupted(final InterruptedException e) {
          *   to the user
          */
         public void moveToState(final QueryState newState, final Exception ex) {
    +      // if the current thread is the foreman thread, throw an exception
    +      // otherwise the foreman will be blocked forever on acceptExternalEvents
    +      if (myThreadRef == Thread.currentThread()) {
    --- End diff --
    
    Well, Foreman does not call moveToState in common path but when it fails then it could. My point is instead of hacking the method to throw an exception if the thread is foreman we should release the latch and handle the exception in foreman#run. What I mean is
    
    Foreman#run() { 
    ```
    try {
      do sth
    } catch(ex) {
      release latch 
      handleException(ex)
    } finally {
      ...
    }
    ```
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: DRILL-4676: Foreman.moveToState can block fore...

Posted by adeneche <gi...@git.apache.org>.
Github user adeneche commented on the pull request:

    https://github.com/apache/drill/pull/503#issuecomment-219571472
  
    This is an alternative change that removes the blocking latch from Foreman. There is still a possibility for the foreman thread to block forever in case of a bug but this should no longer cause rpc threads to block as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---