Yes, that is correct. 'taskmanager.memory.process.size' is the most
recommended.

Thank you~

Xintong Song



On Fri, Jun 12, 2020 at 10:59 PM Clay Teeter <clay.tee...@maalka.com> wrote:

> Ok, this is great to know.  So in my case; I have a k8 pod that has a
> limit of 4Gb.  I should remove the -Xmx and add one of these -D parameters.
>
> * taskmanager.memory.flink.size
> * *taskmanager.memory.process.size.   <- Probably this one*
> * taskmanager.memory.task.heap.size and taskmanager.memory.managed.size
>
> So that i don't run into pod memory quotas
>
>
>
>
>
>
> On Fri, Jun 12, 2020 at 11:12 AM Xintong Song <tonysong...@gmail.com>
> wrote:
>
>> I would suggest not to set -Xmx.
>>
>> Flink will always calculate the JVM heap size from the configuration and
>> set a proper -Xmx.
>> If you manually set -Xmx that overwrites the one Flink calculated, it
>> might result in unpredictable behaviors.
>>
>>
>> Please refer to this document[1]. In short, you could leverage the
>> configuration option "taskmanager.memory.task.heap.size", and an additional
>> constant framework overhead will be added to this value for -Xmx.
>>
>>
>> Thank you~
>>
>> Xintong Song
>>
>>
>> [1]
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_detail.html#jvm-parameters
>>
>> On Fri, Jun 12, 2020 at 4:50 PM Clay Teeter <clay.tee...@maalka.com>
>> wrote:
>>
>>> Thank you Xintong, while tracking down the existence of
>>> bash-java-utils.jar I found a bug in my CI scripts that incorrectly built
>>> the wrong version of flink.  I fixed this and then added a -Xmx value.
>>>
>>> env:
>>>           - name: FLINK_ENV_JAVA_OPTS
>>>             value: "-Xmx{{ .Values.analytics.flink.taskManagerHeapSize
>>> }}"
>>>
>>>
>>> It's running perfectly now!
>>>
>>> Thank you again,
>>> Clay
>>>
>>>
>>> On Fri, Jun 12, 2020 at 5:13 AM Xintong Song <tonysong...@gmail.com>
>>> wrote:
>>>
>>>> Hi Clay,
>>>>
>>>> Could you verify the "taskmanager.sh" used is the same script shipped
>>>> with Flink-1.10.1? Or a custom script is used? Also, does the jar file
>>>> "bash-java-utils.jar" exist in your Flink bin directory?
>>>>
>>>> In Flink 1.10, the memory configuration for a TaskManager works as
>>>> follows.
>>>>
>>>>    - "taskmanager.sh" executes "bash-java-utils.jar" for the memory
>>>>    calculations
>>>>    - "bash-java-utils.jar" will read your "flink-conf.yaml" and all
>>>>    the "-D" arguments, and calculate memory sizes accordingly
>>>>    - "bash-java-utils.jar" will then return the memory calculation
>>>>    results as two strings, for JVM parameter ("-Xmx", "-Xms", etc.) and
>>>>    dynamic configurations ("-D") respectively
>>>>    - At this step, all the detailed memory sizes should be determined
>>>>       - That means, even for memory sizes not configured by you, there
>>>>       should be an exact value generated in the returned dynamic 
>>>> configuration
>>>>       - That also means, for memory components configured in ranges
>>>>       (e.g., network memory configured through a pair of [min, max]),
>>>>       a deterministic value should be decided and both min/max 
>>>> configuration
>>>>       options should already been overwrite to that value
>>>>    - "taskmanager.sh" starts the task manager JVM process with the
>>>>    returned JVM parameters, and passes the dynamic configurations as 
>>>> arguments
>>>>    into the task manager process. These dynamic configurations will be 
>>>> read by
>>>>    Flink task manager so that memory will be managed accordingly.
>>>>
>>>> Flink task manager expects all the memory configurations are already
>>>> set (thus network min/max should have the same value) before it's started.
>>>> In your case, it seems such configurations are missing. Same for the cpu
>>>> cores.
>>>>
>>>> Thank you~
>>>>
>>>> Xintong Song
>>>>
>>>>
>>>>
>>>> On Fri, Jun 12, 2020 at 12:58 AM Clay Teeter <clay.tee...@maalka.com>
>>>> wrote:
>>>>
>>>>> Hi flink fans,
>>>>>
>>>>> I'm hoping for an easy solution.  I'm trying to upgrade my 9.3 cluster
>>>>> to flink 10.1, but i'm running into memory configuration errors.
>>>>>
>>>>> Such as:
>>>>> *Caused by:
>>>>> org.apache.flink.configuration.IllegalConfigurationException: The network
>>>>> memory min (64 mb) and max (1 gb) mismatch, the network memory has to be
>>>>> resolved and set to a fixed value before task executor starts*
>>>>>
>>>>> *Caused by:
>>>>> org.apache.flink.configuration.IllegalConfigurationException: The required
>>>>> configuration option Key: 'taskmanager.cpu.cores' , default: null 
>>>>> (fallback
>>>>> keys: []) is not set*
>>>>>
>>>>> I was able to fix a cascade of errors by explicitly setting these
>>>>> values:
>>>>>
>>>>> taskmanager.memory.managed.size: {{
>>>>> .Values.analytics.flink.taskManagerManagedSize }}
>>>>> taskmanager.memory.task.heap.size: {{
>>>>> .Values.analytics.flink.taskManagerHeapSize }}
>>>>> taskmanager.memory.jvm-metaspace.size: 500m
>>>>> taskmanager.cpu.cores: 4
>>>>>
>>>>> So, the documentation implies that flink will default many of these
>>>>> values, however my 101. cluster doesn't seem to be doing this.  9.3, 
>>>>> worked
>>>>> great!
>>>>>
>>>>> Do I really have to set all the memory (even network) values?  If not,
>>>>> what am I missing?
>>>>>
>>>>> If i do have to set all the memory parameters, how do I resolve "The
>>>>> network memory min (64 mb) and max (1 gb) mismatch"?
>>>>>
>>>>>
>>>>> My cluster runs standalone jobs on kube
>>>>>
>>>>> flnk-config.yaml:
>>>>>     state.backend: rocksdb
>>>>>     state.backend.incremental: true
>>>>>     state.checkpoints.num-retained: 1
>>>>>     taskmanager.memory.managed.size: {{
>>>>> .Values.analytics.flink.taskManagerManagedSize }}
>>>>>     taskmanager.memory.task.heap.size: {{
>>>>> .Values.analytics.flink.taskManagerHeapSize }}
>>>>>     taskmanager.memory.jvm-metaspace.size: 500m
>>>>>     taskmanager.cpu.cores: 4
>>>>>     taskmanager.numberOfTaskSlots: {{
>>>>> .Values.analytics.task.numberOfTaskSlots }}
>>>>>     parallelism.default: {{ .Values.analytics.flink.parallelism }}
>>>>>
>>>>>
>>>>> JobManger:
>>>>>         command: ["/opt/flink/bin/standalone-job.sh"]
>>>>>         args: ["start-foreground", "-j={{
>>>>> .Values.analytics.flinkRunnable }}",  ...
>>>>>
>>>>> TakManager
>>>>>         command: ["/opt/flink/bin/taskmanager.sh"]
>>>>>         args: [
>>>>>           "start-foreground",
>>>>>           "-Djobmanager.rpc.address=localhost",
>>>>>           "-Dmetrics.reporter.prom.port=9430"]
>>>>>
>>>>>
>>>>>
>>>>>

Reply via email to