An update: The extra memory that is being acquired/kept after each pipeline run seems to be coming from off-heap space, especially using `Unsafe.allocateMemory`. I have added some notes here <https://github.com/google/fhir-data-pipes/issues/777#issuecomment-1703142297> but the TL;DR; is that adding a GC hint at the end of each pipeline run mitigates the issue significantly. The idea of adding the GC hint came from FLINK-15758 <https://issues.apache.org/jira/browse/FLINK-15758> so I am not sure if that is still a problem. Our code is open-source and the bandaid fix we have added is this PR <https://github.com/google/fhir-data-pipes/pull/802> (the `System.gc()` is the main relevant part).
Any insights from someone who knows the internals of Flink is appreciated. -B On Wed, Aug 30, 2023 at 3:47 AM Chandrashekar Sankarapu < sankar...@google.com> wrote: > Hi Team, > > We have a data pipeline which is built using Apache Beam SDK > <https://beam.apache.org/> and we use Apache Flink Runner > <https://beam.apache.org/documentation/runners/flink/> to execute Beam > pipelines. We use the local embedded execution mode of Flink for running > the pipelines. > > Currently, we are running into an issue where in the batch pipeline can be > triggered multiple times and each time the pipeline is triggered the Flink > creates a miniCluster in local execution mode, runs the job and destroys > the miniCluster once the job is completed. When the batch job is triggered > multiple times we observe the application process Resident Set Size(RSS) > memory keeps increasing (approximately increases by a value set for > parameter 'taskmanager.memory.network.max') with each run and is not > released, eventually leading to crash of the docker container (container > memory is limited) in which this is deployed. However, when we checked the > JVM memory using tools like JConsole, jcmd etc does not show any increase > and is always with the Xmx value set. > > We analysed the heap dump of the application but it didn't show any memory > leaks. > > Has anyone faced this issue? Any pointers are appreciated. > > Thanks, > Chandra >