flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Fabian Hueske <fhue...@gmail.com>
Subject Re: off-heap size feature request
Date Wed, 16 Mar 2016 13:45:22 GMT
taskmanager.heap.mb defines the total amount of memory used by a task
manager. The "heap" part of the parameter name originates from the time
when Flink could not allocate off-heap and I agree it is confusing.
Nonetheless, we decided to keep it for backward compatibility.

The memory fraction is defined wrt taskmanager.heap.mb, i.e. , the total TM
memory. If it is defined as 0.6 and off-heap enabled, 60% of the total size
go into off-heap memory and 40% go into the JVM heap. The related code is
in the TM start script: ./bin/taskmanager.sh. If this is not working
correctly, there is bug.

Thanks for you suggestions regarding the config parameters.
With release 1.0 we committed to backward compatibility. So until Flink 2.0
we cannot remove parameters or change parameter semantics. I believe adding
new parameters for the same knobs would make parameter reasoning very hard
and might confuse users even more,


2016-03-16 14:30 GMT+01:00 Ovidiu-Cristian MARCU <
ovidiu-cristian.marcu@inria.fr>:

> Updating the documentation will partially remove the confusion.
> Heap memory is JVM managed while off-heap is outside JVM and I would
> define its size separately.
>
> In my case, I see memory usage going up to full utilisation on a node
> where full RAM available is 128GB in the following conditions:
> taskmanager.heap.mb = 12*1024
> taskmanager.memory.off-heap: true
> taskmanager.memory.fraction: 0.6
>
> So the memory fraction will be applied to the free memory and in the
> absence of the taskmanager.memory.size parameter, the fraction is defining
> the off-heap size.
> Everything else in the memory usage is network buffers I guess
> (taskmanager.network.numberOfBuffers *
> taskmanager.network.bufferSizeInBytes), along with OS cache.
>
> I would like parameters like:
> taskmanager.off-heap.size or taskmanager.off-heap.fraction
> taskmanager.off-heap.enabled true or false
> and same for heap.
>
> Thanks for clarification.
>
> Best,
> Ovidiu
>
>
> On 16 Mar 2016, at 13:43, Fabian Hueske <fhueske@gmail.com> wrote:
>
> Oh yes, good point! The documentation needs to be updated to something
> like:
>
> "The amount of memory (in megabytes) that the task manager reserves
> on-heap or off-heap (depending on taskmanager.memory.off-heap) for
> sorting, hash tables, and caching of intermediate results. If unspecified
> (-1), the memory manager will take a fixed ratio with respect to the size
> of the task manager JVM as specified by taskmanager.memory.fraction."
>
> I will open a JIRA to fix the documentation.
>
> If you do the following
> taskmanager.heap.mb: 4096
> taskmanager.memory.size: 2048
> taskmanager.memory.off-heap: true
>
> The TaskManager will be started with a 2GB (4096MB-2048MB) JVM and
> allocate (2048MB) as off-heap memory. Hence, the overall process size will
> be roughly 4GB. The parameter name "taskmanager.heap.mb" is a bit confusing
> in case of off-heap memory usage, because it does not define this size of
> the heap but of the overall process.
>
> Hope this helps,
> Fabian
>
>
>
> 2016-03-16 12:40 GMT+01:00 Ovidiu-Cristian MARCU <
> ovidiu-cristian.marcu@inria.fr>:
>
>> Thanks!
>> I will try this one: taskmanager.memory.size. So I should expect this
>> will be the off-heap memory size, right?
>> I am using taskmanager.heap.mb=some value, taskmanager.memory.off-heap:
>> true
>> Memory usage goes up to 99%.
>>
>> The documentation is confusing:
>>
>> taskmanager.memory.size: The amount of memory (in megabytes) that *the
>> task manager reserves on the JVM’s heap* space for sorting, hash tables,
>> and caching of intermediate results. If unspecified (-1), the memory
>> manager will take a fixed ratio of the heap memory available to the JVM, as
>> specified by taskmanager.memory.fraction.  [1]
>>
>> [1]
>> https://ci.apache.org/projects/flink/flink-docs-release-1.0/setup/config.html#managed-memory
>>
>> Best,
>> Ovidiu
>>
>> On 16 Mar 2016, at 12:13, Fabian Hueske <fhueske@gmail.com> wrote:
>>
>> Hi Ovidiu,
>>
>> the parameters to configure the amount of managed memory (taskmanager.memory.size,
>> taskmanager.memory.fraction) are valid for on and off-heap memory.
>>
>> Have you tried these parameters and didn't they work as expected?
>>
>> Best, Fabian
>>
>>
>> 2016-03-16 11:43 GMT+01:00 Ovidiu-Cristian MARCU <
>> ovidiu-cristian.marcu@inria.fr>:
>>
>>> Hi,
>>>
>>> Is it possible to add a parameter off-heap.size for the task manager
>>> off-heap memory [1]?
>>>
>>> It is not possible to limit the off-heap memory size, at least I found
>>> nothing in the documentation.
>>>
>>> [1]
>>> https://ci.apache.org/projects/flink/flink-docs-release-1.0/setup/config.html#managed-memory
>>>
>>> Best,
>>> Ovidiu
>>>
>>
>>
>>
>
>

Mime
View raw message