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

ASF GitHub Bot commented on FLINK-3322:
---------------------------------------

GitHub user ramkrish86 opened a pull request:

    https://github.com/apache/flink/pull/2495

    FLINK-3322 - Make sorters to reuse the memory pages allocated for iterative 
tasks

    Thanks for contributing to Apache Flink. Before you open your pull request, 
please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your 
pull request. For more information and/or questions please refer to the [How To 
Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful 
description of your changes.
    
    - [ ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira 
title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the 
JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [ ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis 
build has passed
    
    
    This is part1 for FLINK-3322 where only the Sorters are made to reuse the 
memory pages. As @ggevay  pointed out we have to handle the iterators also 
where the memory pages are allocated. I have a seperate PR for that because 
that involves touching lot of places. But am open to feedback here. It is fine 
with me to combine both also but it was making the changes much bigger. 
    I would like to get the feed back here on this apporach. 
    Here a SorterMemoryAllocator is now passed to the UnilateralSortMergers. 
That will allocate the required memory pages and it will allocate the required 
read, write and large buffers. As per the existing logic the buffers will be 
released. But if the task is an iterative task we wait for the tasks to be 
released until a close or termination call happens for the iterative task. 
    In case of pages that were grabbed in between for keysort or record sort 
those will be put back to the respective pages so that we have the required 
number of pages through out the life cycle of the iterative task.
    
    As said this is only part 1. We need to address the iterators also. But 
that according to me touches more places. I have done the changes for that but 
it is not in a shape to be pushed as a PR but am open to feed back here. Thanks 
all. 

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

    $ git pull https://github.com/ramkrish86/flink FLINK-3322_part1

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

    https://github.com/apache/flink/pull/2495.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 #2495
    
----
commit 705ee5294bc5263971c2924a55c9230d72806527
Author: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Date:   2016-09-13T06:33:59Z

    FLINK-3322 - Make sorters to reuse the memory pages allocated for
    iterative tasks

----


> MemoryManager creates too much GC pressure with iterative jobs
> --------------------------------------------------------------
>
>                 Key: FLINK-3322
>                 URL: https://issues.apache.org/jira/browse/FLINK-3322
>             Project: Flink
>          Issue Type: Bug
>          Components: Local Runtime
>    Affects Versions: 1.0.0
>            Reporter: Gabor Gevay
>            Priority: Critical
>             Fix For: 1.0.0
>
>         Attachments: FLINK-3322.docx
>
>
> When taskmanager.memory.preallocate is false (the default), released memory 
> segments are not added to a pool, but the GC is expected to take care of 
> them. This puts too much pressure on the GC with iterative jobs, where the 
> operators reallocate all memory at every superstep.
> See the following discussion on the mailing list:
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Memory-manager-behavior-in-iterative-jobs-tt10066.html
> Reproducing the issue:
> https://github.com/ggevay/flink/tree/MemoryManager-crazy-gc
> The class to start is malom.Solver. If you increase the memory given to the 
> JVM from 1 to 50 GB, performance gradually degrades by more than 10 times. 
> (It will generate some lookuptables to /tmp on first run for a few minutes.) 
> (I think the slowdown might also depend somewhat on 
> taskmanager.memory.fraction, because more unused non-managed memory results 
> in rarer GCs.)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to