[ 
https://issues.apache.org/jira/browse/FLINK-38212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Grzegorz Liter updated FLINK-38212:
-----------------------------------
    Environment:     (was: Flink 2.1.0 running in Application mode with Flink 
Operator 1.12.1.

Memory and savepoint related settings:
{code:java}
env.java.opts.taskmanager: ' -XX:+UnlockExperimentalVMOptions 
-XX:+UseStringDeduplication
      -XX:+AlwaysPreTouch  -XX:G1HeapRegionSize=16m 
-Xlog:gc*:file=/tmp/gc.log:time,uptime,level,tags
      -XX:SurvivorRatio=6 -XX:G1NewSizePercent=40
execution.checkpointing.max-concurrent-checkpoints: "1"
execution.checkpointing.snapshot-compression: "true"
fs.s3a.aws.credentials.provider: 
com.amazonaws.auth.WebIdentityTokenCredentialsProvider
fs.s3a.block.size: 
fs.s3a.experimental.input.fadvise: sequential
fs.s3a.path.style.access: "true" 
state.backend.incremental: "true"
state.backend.type: rocksdb 
state.checkpoints.dir: s3p://bucket/checkpoints
state.savepoints.dir: s3p://bucket/savepoints 
taskmanager.memory.jvm-overhead.fraction: "0.1"
taskmanager.memory.jvm-overhead.max: 6g
taskmanager.memory.managed.fraction: "0.4"
taskmanager.memory.network.fraction: "0.05"
taskmanager.network.memory.buffer-debloat.enabled: "true"
taskmanager.numberOfTaskSlots: "12"
...
resource:
  memory: 16g{code}
 )

> OOM during savepoint caused by potential memory leak issue in RocksDB related 
> to jemalloc
> -----------------------------------------------------------------------------------------
>
>                 Key: FLINK-38212
>                 URL: https://issues.apache.org/jira/browse/FLINK-38212
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.20.2, 2.1.0
>            Reporter: Grzegorz Liter
>            Priority: Major
>         Attachments: image-2025-08-07-17-14-03-023.png, 
> image-2025-08-07-17-15-11-647.png
>
>
> I am running a job with snapshot size about ~17 GB with compression enabled. 
> I have observed that savepoints often fails due to TM getting killed by 
> Kubernetes due to exceeding memory limit on pod that had 30 GB of memory 
> limit assigned.
> Flink metrics nor detailed VM metrics taken with `jcmd <PID> VM.native_memory 
> detail` does not indicate any unusual memory increase. Consumed memory is 
> visible only in Kubernetes metrics and RSS.
> When enough memory set (+ potentially setting enough jvm overhead) to leave 
> some breathing room one snapshot could be taken but taking subsequent full 
> snapshots reliably leads to OOM.
> This documentation: 
> [switching-the-memory-allocator|https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/resource-providers/standalone/docker/#switching-the-memory-allocator]
>  have lead me to trying 
> {code:java}
> MALLOC_ARENA_MAX=1
> DISABLE_JEMALLOC=true {code}
> This configuration helped to make savepoint reliably pass without OOM. I have 
> trying setting only one of each options at once but that was not fixing the 
> issue.
> I also tried downscaling pod down to 16 GB of memory and with these options 
> savepoint was reliably created without any issue. Without them every 
> savepoint fails.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to