drill-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (DRILL-6125) PartitionSenderRootExec can leak memory because close method is not synchronized
Date Fri, 09 Mar 2018 19:00:00 GMT

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

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

Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1105#discussion_r173537455
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
---
    @@ -488,47 +548,74 @@ void receiverFinished(FragmentHandle handle) {
           sendEvent(new FragmentEvent(EventType.RECEIVER_FINISHED, handle));
         }
     
    +    /**
    +     * Tell the {@link FragmentEventProcessor} not to process anymore events. This keeps
stray cancellation requests
    +     * from being processed after the root has finished running and interrupts in the
root thread have been cleared.
    +     */
    +    public void terminate() {
    +      terminate.set(true);
    +    }
    +
         @Override
         protected void processEvent(FragmentEvent event) {
    +      if (event.type.equals(EventType.RECEIVER_FINISHED)) {
    +        // Finish request
    +        if (terminate.get()) {
    +          // We have already recieved a cancellation or we have terminated the event
processor. Do not process anymore finish requests.
    +          return;
    +        }
    +      } else {
    +        // Cancel request
    +        if (!terminate.compareAndSet(false, true)) {
    +          // We have already received a cancellation or we have terminated the event
processor. Do not process anymore cancellation requests.
    +          // This prevents the root thread from being interrupted at an inappropriate
time.
    +          return;
    +        }
    +      }
    +
           switch (event.type) {
             case CANCEL:
    -          /*
    -           * We set the cancel requested flag but the actual cancellation is managed
by the run() loop, if called.
    -           */
    +          // 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. We
    -           * synchronize here to ensure we don't accidentally create a race condition
where we interrupt the close out
    -           * procedure of the main thread.
    -          */
    -          synchronized (myThreadRef) {
    -            final Thread myThread = myThreadRef.get();
    -            if (myThread != null) {
    -              logger.debug("Interrupting fragment thread {}", myThread.getName());
    -              myThread.interrupt();
    -            }
    -          }
    +          // The root was started so we have to interrupt it in case it is performing
a blocking operation.
    +          killThread();
    +          terminate.set(true);
    --- End diff --
    
    Doh! Thanks for catching


> PartitionSenderRootExec can leak memory because close method is not synchronized
> --------------------------------------------------------------------------------
>
>                 Key: DRILL-6125
>                 URL: https://issues.apache.org/jira/browse/DRILL-6125
>             Project: Apache Drill
>          Issue Type: Bug
>    Affects Versions: 1.13.0
>            Reporter: Timothy Farkas
>            Assignee: Timothy Farkas
>            Priority: Minor
>             Fix For: 1.14.0
>
>
> PartitionSenderRootExec creates a PartitionerDecorator and saves it in the *partitioner*
field. The creation of the partitioner happens in the createPartitioner method. This method
get's called by the main fragment thread. The partitioner field is accessed by the fragment
thread during normal execution but it can also be accessed by the receivingFragmentFinished
method which is a callback executed by the event processor thread. Because multiple threads
can access the partitioner field synchronization is done on creation and on when receivingFragmentFinished.
However, the close method can also be called by the event processor thread, and the close
method does not synchronize before accessing the partitioner field. Since synchronization
is not done the event processor thread may have an old reference to the partitioner when a
query cancellation is done. Since it has an old reference the current partitioner can may
not be cleared and a memory leak may occur.



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

Mime
View raw message