Hi Alexey,

You may want to double check if `state.backend.rocksdb.memory.managed` is
configured to `true`. (This should be `true` by default.)

Another question that may or may not be related. I noticed that you have
configured 128MB task off-heap memory, which IIRC the default should be 0.
Could you share what that is for?

Thank you~

Xintong Song



On Tue, Feb 15, 2022 at 12:10 PM Alexey Trenikhun <yen...@msn.com> wrote:

> Hello,
> We use RocksDB, but there is no problem with Java heap, which is limited
> by 3.523gb, the problem with total container memory. The pod is killed
> not due OutOfMemoryError,  but because total container memory exceeds 10gb
>
> Thanks,
> Alexey
> ------------------------------
> *From:* Caizhi Weng <tsreape...@gmail.com>
> *Sent:* Monday, February 14, 2022 6:42:05 PM
> *To:* Alexey Trenikhun <yen...@msn.com>
> *Cc:* Flink User Mail List <user@flink.apache.org>
> *Subject:* Re: TM OOMKilled
>
> Hi!
>
> Heap memory usage depends heavily on your job and your state backend.
> Which state backend are you using and if possible could you share your user
> code or explain what operations your job is doing?
>
> Alexey Trenikhun <yen...@msn.com> 于2022年2月15日周二 05:17写道:
>
> Hello,
> We run Flink 1.13.5 job in app mode in Kubernetes, 1 JM and 1 TM, we also
> have Kubernetes cron job which takes savepoint every 2 hour (14 */2 * * *),
> once in while (~1 per 2 days) TM is OOMKilled, suspiciously it happens on
> even hours ~4 minutes after savepoint start (e.g. 12:18, 4:18) but I don't
> see failed save points, so I assume OOM happens right after savepoint
> taken. However OOMKilled doesn't happen on every save point, so maybe this
> is a random correlation.
> I've reserved 2G for JVM overhead, but somehow it is not enough ? Any
> known issues with memory and savepoints? Any suggestions how to
> troubleshoot this?
>
>  Final TaskExecutor Memory configuration:
>    Total Process Memory:          10.000gb (10737418240 bytes)
>      Total Flink Memory:          7.547gb (8103395328 bytes)
>        Total JVM Heap Memory:     3.523gb (3783262149 bytes)
>          Framework:               128.000mb (134217728 bytes)
>          Task:                    3.398gb (3649044421 bytes)
>        Total Off-heap Memory:     4.023gb (4320133179 bytes)
>          Managed:                 3.019gb (3241358179 bytes)
>          Total JVM Direct Memory: 1.005gb (1078775000 bytes)
>            Framework:             128.000mb (134217728 bytes)
>            Task:                  128.000mb (134217728 bytes)
>            Network:               772.800mb (810339544 bytes)
>      JVM Metaspace:               256.000mb (268435456 bytes)
>      JVM Overhead:                2.203gb (2365587456 bytes)
>
> Thanks,
> Alexey
>
>

Reply via email to