Re: JobManager seems to be leaking temporary jar files

2021-01-28 Thread Chesnay Schepler
Code-wise it appears that thing have gotten simpler and we can use use a URLClassLoader within PackagedProgram. We probably won't get around a dedicated close() method on the PackagedProgram. I think in FLINK-21164 I think have identified the right places to issue this call within the jar ha

Re: JobManager seems to be leaking temporary jar files

2021-01-27 Thread Maciek Próchniak
Hi Chesnay, thanks for reply. I wonder if FLINK-21164 will help without FLINK-9844 - if the jar file is not closed, it won't be successfully deleted? As for FLINK-9844 - I understand that having code like if (userClassLoader instanceof Closeable) { ((Closeable) userClassloader).close() } i

Re: JobManager seems to be leaking temporary jar files

2021-01-27 Thread Chesnay Schepler
The problem of submitted jar files not being closed is a known one: https://issues.apache.org/jira/browse/FLINK-9844 IIRC it's not exactly trivial to fix since class-loading is involved. It's not strictly related to the REST API; it also occurs in the CLI but is less noticeable since jars are us

Re: JobManager seems to be leaking temporary jar files

2021-01-26 Thread Maciek Próchniak
Hi Matthias, I think the problem lies somewhere in JarRunHandler, as this is the place where the files are created. I think these are not the files that are managed via BlobService, as they are not stored in BlobService folders (I made experiment changing default BlobServer folders). It se

Re: JobManager seems to be leaking temporary jar files

2021-01-26 Thread Matthias Pohl
Hi Maciek, my understanding is that the jars in the JobManager should be cleaned up after the job is terminated (I assume that your jobs successfully finished). The jars are managed by the BlobService. The dispatcher will trigger the jobCleanup in [1] after job termination. Are there any suspicious

JobManager seems to be leaking temporary jar files

2021-01-25 Thread Maciek Próchniak
Hello, in our setup we have: - Flink 1.11.2 - job submission via REST API (first we upload jar, then we submit multiple jobs with it) - additional jars embedded in lib directory of main jar (this is crucial part) When we submit jobs this way, Flink creates new temp jar files via Packaged