I've just tested the following code in a java class and the property (-Dcom.mysql.cj.disableAbandonedConnectionCleanup=true) is read correctly and the abandonedConnectionCleanupDisabled does not initialize the cleanupThreadExecutorService (that in my other test was causing a dynamic classloading memory leak):
try { Class.forName(drivername); Connection dbConn = DriverManager.getConnection(dbURL, username, password); System.out.println("OK"); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } catch (ClassNotFoundException cnfe) { throw new IllegalArgumentException("JDBC-Class not found. - " + cnfe.getMessage(), cnfe); } This is the output of ps aux command (I can also notice that the taskmanager parameters are created using a space after the -D option that I don't know if it intended or not): flink 23904 59.5 1.1 5488264 380396 pts/5 Sl 10:41 0:08 java -Xmx536870902 -Xms536870902 -XX:MaxDirectMemorySize=268435458 -XX:MaxMetaspaceSize=268435456 -Dcom.mysql.disableAbandonedConnectionCleanup=true -Dlog.file=/opt/flink/flink-1.11.0/log/flink-flink-taskexecutor-0-flavio-ThinkPad-P50.log -Dlog4j.configuration=file:/opt/flink/flink-1.11.0/conf/log4j.properties -Dlog4j.configurationFile=file:/opt/flink/flink-1.11.0/conf/log4j.properties -Dlogback.configurationFile=file:/opt/flink/flink-1.11.0/conf/logback.xml -classpath /opt/flink/flink-1.11.0/lib/flink-csv-1.11.0.jar:/opt/flink/flink-1.11.0/lib/flink-json-1.11.0.jar:/opt/flink/flink-1.11.0/lib/flink-shaded-zookeeper-3.4.14.jar:/opt/flink/flink-1.11.0/lib/flink-table_2.12-1.11.0.jar:/opt/flink/flink-1.11.0/lib/flink-table-blink_2.12-1.11.0.jar:/opt/flink/flink-1.11.0/lib/log4j-1.2-api-2.12.1.jar:/opt/flink/flink-1.11.0/lib/log4j-api-2.12.1.jar:/opt/flink/flink-1.11.0/lib/log4j-core-2.12.1.jar:/opt/flink/flink-1.11.0/lib/log4j-slf4j-impl-2.12.1.jar:/opt/flink/flink-1.11.0/lib/mariadb-java-client-2.6.0.jar:/opt/flink/flink-1.11.0/lib/mssql-jdbc-7.4.1.jre11.jar:/opt/flink/flink-1.11.0/lib/mysql-connector-java-8.0.22.jar:/opt/flink/flink-1.11.0/lib/flink-dist_2.12-1.11.0.jar::: org.apache.flink.runtime.taskexecutor.TaskManagerRunner --configDir /opt/flink/flink-1.11.0/conf -D taskmanager.memory.framework.off-heap.size=134217728b -D taskmanager.memory.network.max=134217730b -D taskmanager.memory.network.min=134217730b -D taskmanager.memory.framework.heap.size=134217728b -D taskmanager.memory.managed.size=536870920b -D taskmanager.cpu.cores=1.0 -D taskmanager.memory.task.heap.size=402653174b -D taskmanager.memory.task.off-heap.size=0b The mysql dependency in the maven project is: <dependency> <groupId>mysql</groupId> <artifactId>mysql-connector-java</artifactId> <version>8.0.22</version> <scope>provided</scope> </dependency> On Fri, Nov 20, 2020 at 10:07 AM Flavio Pompermaier <pomperma...@okkam.it> wrote: > no no I didn't relocate any class related to jdbc > > Il ven 20 nov 2020, 10:02 Arvid Heise <ar...@ververica.com> ha scritto: > >> I was particularly asking if you relocate classes. Since the property >> name looks like a class name, it could have been changed as well. Could you >> check the value of >> PropertyDefinitions.SYSP_disableAbandonedConnectionCleanup in your final >> jar? >> >> On Fri, Nov 20, 2020 at 9:35 AM Flavio Pompermaier <pomperma...@okkam.it> >> wrote: >> >>> the mysql connector is put in the client classpath and in the Flink lib >>> dir. When i debugged remotely the AbandonedConnectionCleanupThread was >>> initialized at the first run of the job by the taskmamager. Today I'll try >>> to run the mysql connector in a standalone java app to see if the property >>> is read correctly or not. >>> >>> Il ven 20 nov 2020, 07:52 Arvid Heise <ar...@ververica.com> ha scritto: >>> >>>> Hi Flavio, >>>> >>>> if it arrives in the java process then you are doing everything right >>>> already (or almost). >>>> >>>> Are you shading the mysql connector? I'm suspecting that the property >>>> also get shaded then. You could decompile your jar to be sure. Have you >>>> verified that this is working as intended without Flink? >>>> >>>> On Thu, Nov 19, 2020 at 9:19 PM Flavio Pompermaier < >>>> pomperma...@okkam.it> wrote: >>>> >>>>> the properties arrives to the task manager because I can see them in >>>>> the java process (using ps aux)..or donyoubmean some special line of code? >>>>> >>>>> Il gio 19 nov 2020, 20:53 Arvid Heise <ar...@ververica.com> ha >>>>> scritto: >>>>> >>>>>> Hi Flavio, >>>>>> >>>>>> you are right, all looks good. >>>>>> >>>>>> Can you please verify if the properties arrived at the task manager >>>>>> in the remote debugger session? For example, you could check the >>>>>> JVisualVM >>>>>> Overview tab. >>>>>> >>>>>> On Thu, Nov 19, 2020 at 8:38 PM Flavio Pompermaier < >>>>>> pomperma...@okkam.it> wrote: >>>>>> >>>>>>> At the moment I use a standalone cluster, isn't using env.java.opts >>>>>>> the right way to do it? >>>>>>> >>>>>>> Il gio 19 nov 2020, 20:11 Arvid Heise <ar...@ververica.com> ha >>>>>>> scritto: >>>>>>> >>>>>>>> Hi Flavio, >>>>>>>> >>>>>>>> -D afaik passes only the system property to the entry point (client >>>>>>>> or jobmanager depending on setup), while you probably want to have it >>>>>>>> on >>>>>>>> the task managers. >>>>>>>> >>>>>>>> The specific options to pass it to the task managers depend on the >>>>>>>> way you deploy. -yD for yarn for example. For docker or k8s, you would >>>>>>>> use >>>>>>>> env. >>>>>>>> >>>>>>>> On Wed, Nov 18, 2020 at 10:20 PM Flavio Pompermaier < >>>>>>>> pomperma...@okkam.it> wrote: >>>>>>>> >>>>>>>>> Hi to all, >>>>>>>>> while trying to solve a leak with dynamic class loading I found out >>>>>>>>> that mysql connector creates an AbandonedConnectionCleanupThread >>>>>>>>> that >>>>>>>>> is retained in the ChildFirstClassLoader..from version 8.0.22 >>>>>>>>> there's >>>>>>>>> the possibility to inhibit this thread passing the system property >>>>>>>>> com.mysql.disableAbandonedConnectionCleanup=true [1] (I put the >>>>>>>>> mysql >>>>>>>>> jar in the lib folder). >>>>>>>>> >>>>>>>>> I tried to set in the flink-conf.yml >>>>>>>>> env.java.opts: "-Dcom.mysql.disableAbandonedConnectionCleanup=true" >>>>>>>>> >>>>>>>>> but the property does not produce the desired effect in the static >>>>>>>>> section of such a thread [2] (I verified that attaching the remote >>>>>>>>> debugger to the task manager). >>>>>>>>> >>>>>>>>> How can I fix this problem? >>>>>>>>> >>>>>>>>> [1] >>>>>>>>> https://dev.mysql.com/doc/relnotes/connector-j/8.0/en/news-8-0-22.html >>>>>>>>> [2] >>>>>>>>> public class AbandonedConnectionCleanupThread implements Runnable >>>>>>>>> {8898 >>>>>>>>> 7324 0768 >>>>>>>>> private static boolean abandonedConnectionCleanupDisabled = >>>>>>>>> >>>>>>>>> Boolean.getBoolean(PropertyDefinitions.SYSP_disableAbandonedConnectionCleanup); >>>>>>>>> >>>>>>>>> static { >>>>>>>>> if (abandonedConnectionCleanupDisabled) { >>>>>>>>> cleanupThreadExecutorService = null; >>>>>>>>> } else { >>>>>>>>> cleanupThreadExecutorService = >>>>>>>>> Executors.newSingleThreadExecutor(r -> {....} >>>>>>>>> } >>>>>>>>> } >>>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> -- >>>>>>>> >>>>>>>> Arvid Heise | Senior Java Developer >>>>>>>> >>>>>>>> <https://www.ververica.com/> >>>>>>>> >>>>>>>> Follow us @VervericaData >>>>>>>> >>>>>>>> -- >>>>>>>> >>>>>>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink >>>>>>>> Conference >>>>>>>> >>>>>>>> Stream Processing | Event Driven | Real Time >>>>>>>> >>>>>>>> -- >>>>>>>> >>>>>>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >>>>>>>> >>>>>>>> -- >>>>>>>> Ververica GmbH >>>>>>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B >>>>>>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung >>>>>>>> Jason, Ji (Toni) Cheng >>>>>>>> >>>>>>> >>>>>> >>>>>> -- >>>>>> >>>>>> Arvid Heise | Senior Java Developer >>>>>> >>>>>> <https://www.ververica.com/> >>>>>> >>>>>> Follow us @VervericaData >>>>>> >>>>>> -- >>>>>> >>>>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink >>>>>> Conference >>>>>> >>>>>> Stream Processing | Event Driven | Real Time >>>>>> >>>>>> -- >>>>>> >>>>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >>>>>> >>>>>> -- >>>>>> Ververica GmbH >>>>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B >>>>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, >>>>>> Ji (Toni) Cheng >>>>>> >>>>> >>>> >>>> -- >>>> >>>> Arvid Heise | Senior Java Developer >>>> >>>> <https://www.ververica.com/> >>>> >>>> Follow us @VervericaData >>>> >>>> -- >>>> >>>> Join Flink Forward <https://flink-forward.org/> - The Apache Flink >>>> Conference >>>> >>>> Stream Processing | Event Driven | Real Time >>>> >>>> -- >>>> >>>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >>>> >>>> -- >>>> Ververica GmbH >>>> Registered at Amtsgericht Charlottenburg: HRB 158244 B >>>> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji >>>> (Toni) Cheng >>>> >>> >> >> -- >> >> Arvid Heise | Senior Java Developer >> >> <https://www.ververica.com/> >> >> Follow us @VervericaData >> >> -- >> >> Join Flink Forward <https://flink-forward.org/> - The Apache Flink >> Conference >> >> Stream Processing | Event Driven | Real Time >> >> -- >> >> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >> >> -- >> Ververica GmbH >> Registered at Amtsgericht Charlottenburg: HRB 158244 B >> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji >> (Toni) Cheng >> >