Hi Tuomas,

are you sure that all dependencies have been upgraded to Flink 1.14. Connector dependencies that still reference Flink 1.13 might cause issues.

JdbcBatchingOutputFormat has been refactored in this PR:

https://github.com/apache/flink/pull/16528

I hope this helps.

Regards,
Timo

On 18.12.21 12:30, Tuomas Pystynen wrote:
I am just trying out Flink using images from Docker Hub. My simple insert using JDBC connector to Postgres database fails with this error

Flink SQL> insert into order_counts
 > select customerid, count(*) order_count from orders group by customerid;
[INFO] Submitting SQL update statement to the cluster...
[INFO] SQL update statement has been successfully submitted to the cluster:
Job ID: 5d3b32bd8cc0f11dfe73cbf242793cc9


2021-12-17 20:39:08,975 WARN  org.apache.flink.runtime.taskmanager.Task                    [] - GroupAggregate(groupBy=[customerid], select=[customerid, COUNT(*) AS order_count]) -> NotNullEnforcer(fields=[customerid]) -> Sink: Sink(table=[default_catalog.default_database.order_counts], fields=[customerid, order_count]) (1/1)#0 (12466e41fb377bcd45b3d22aab1cadfd) switched from INITIALIZING to FAILED with failure cause: java.lang.NoClassDefFoundError: org/apache/flink/runtime/util/ExecutorThreadFactory at *org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.open*(JdbcBatchingOutputFormat.java:118) at org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction.open(GenericJdbcSinkFunction.java:49) at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:34) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:100) at org.apache.flink.table.runtime.operators.sink.SinkOperator.open(SinkOperator.java:58) at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:110) at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:711) at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55) at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:687) at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:654) at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958)
at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassNotFoundException: org.apache.flink.runtime.util.ExecutorThreadFactory
at java.net.URLClassLoader.findClass(URLClassLoader.java:387)
at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
... 15 more

After downgrading to Flink 1.13 the insert works fine. Has JdbcBatchingOutputFormat.java been removed by accident or purpose in 1.14 ?

Regards,
Tuomas Pystynen

Reply via email to