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 18:24:00 GMT

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

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

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

    https://github.com/apache/drill/pull/1105#discussion_r173526625
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
---
    @@ -245,19 +306,17 @@ public Void run() throws Exception {
             // we have a heap out of memory error. The JVM in unstable, exit.
             CatastrophicFailure.exit(e, "Unable to handle out of memory condition in FragmentExecutor.",
-2);
           }
    +    } catch (InterruptedException e) {
    +      // Swallow interrupted exceptions since we intentionally interrupt the root when
cancelling a query
    +      logger.trace("Interruped root: {}", e);
         } catch (Throwable t) {
           fail(t);
         } finally {
     
    -      // no longer allow this thread to be interrupted. We synchronize here to make sure
that cancel can't set an
    -      // interruption after we have moved beyond this block.
    -      synchronized (myThreadRef) {
    -        myThreadRef.set(null);
    -        Thread.interrupted();
    -      }
    -
    -      // Make sure the event processor is started at least once
    -      eventProcessor.start();
    +      // Don't process any more termination requests, we are done.
    +      eventProcessor.terminate();
    --- End diff --
    
    Is this terminate() required?


> 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