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

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

sachouche opened a new pull request #1497: DRILL-6410: Fixed memory leak in 
flat Parquet reader
URL: https://github.com/apache/drill/pull/1497
 
 
   **Problem Description**
   - Occasionally, a memory leak is observed within the Parquet reader (flat) 
when query cancellation is invoked
   - I tried to fix this leak in a previous attempt but it seems it is still 
happening
   - Thus far, only QA have been able to observe this issue (and only 
occasionally)
   
   **Analysis**
   - There was a recent breakthrough which gives me hope for addressing this 
issue
   - The leak logged two piece of information: leak size and state of the child 
allocator
   - The state of the child allocator indicated no leak (all allocated bytes 
were released)
   - After code examination, it occurred to me this was happening because the 
Asynchronous Page Reader task was releasing the Drill buffer while the scan 
thread was closing the allocator
   - The code attempts to cancel asynchronous tasks and then release allocated 
buffers, though there is one big caveat: the Java FutureTask.cancel(true) 
doesn't block during the cancellation process; this method merely interrupts 
the asynchronous task and proceeds
   - This means if the asynchronous thread was context switched or doing 
computation (not blocked waiting), then the fragment cleanup logic can close 
the allocator before all resources have been released
   
   **Fix**
   - There are multiple ways to address this issue; chose the simplest and 
least intrusive fix
   - Created a help class that will be in charge of submitting a callable task 
C using an ExecutorService E
   - The helper class will return a wrapper future that will essentially 
decorate the cancellation method and nothing else to ensure no performance 
regressions (or other) are observed
   - Added a test to ensure this logic works as expected
   
   **NOTE**
   - This fix could be extended to include a set of tasks and implement a 
blocking wait (unlike the JDK implementation); the fix would also be efficient 
as it can call cancellation (using no blocking call) and only then perform a 
wait
   - It was felt that providing a non-intrusive & simpler implementation is the 
way to go
    
   

----------------------------------------------------------------
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:
[email protected]


> Memory leak in Parquet Reader during cancellation
> -------------------------------------------------
>
>                 Key: DRILL-6410
>                 URL: https://issues.apache.org/jira/browse/DRILL-6410
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - Parquet
>            Reporter: salim achouche
>            Assignee: salim achouche
>            Priority: Major
>             Fix For: 1.15.0
>
>
> Occasionally, a memory leak is observed within the flat Parquet reader when 
> query cancellation is invoked.



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

Reply via email to