使用flinksql 1.11版本,运行计算好的指标落入mysql,长时间没有数据会报错,导致任务会失败。
针对:https://issues.apache.org/jira/browse/FLINK-16681,已经修复此问题,但是我们使用1.11最新版本,运行一段时间后,发现还是会有此问题,如何解决 -- Sent from: http://apache-flink.147419.n8.nabble.com/ |
JdbcBatchingOutputFormat: for (int i = 0; i <= executionOptions.getMaxRetries(); i++) { try { attemptFlush(); batchCount = 0; break; } catch (SQLException e) { LOG.error("JDBC executeBatch error, retry times = {}", i, e); if (i >= executionOptions.getMaxRetries()) { throw new IOException(e); } try { if (!connection.isValid(CONNECTION_CHECK_TIMEOUT_SECONDS)) { connection = connectionProvider.reestablishConnection(); jdbcStatementExecutor.closeStatements(); jdbcStatementExecutor.prepareStatements(connection); } } catch (Exception excpetion) { LOG.error("JDBC connection is not valid, and reestablish connection failed.", excpetion); throw new IOException("Reestablish JDBC connection failed", excpetion); } try { Thread.sleep(1000 * i); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); throw new IOException("unable to flush; interrupted while doing another attempt", e); } } } 嗯,看起来是这样的。 if (i >= executionOptions.getMaxRetries()) { throw new IOException(e); }这个判断重试的代码应该放在sleep 后面。不然,Caused by: java.io.IOException: java.sql.SQLException: No operations allowed after statement closed. 就没机会重建连接了。 在 2020-12-03 10:36:28,"yanzi" <[hidden email]> 写道: >使用flinksql 1.11版本,运行计算好的指标落入mysql,长时间没有数据会报错,导致任务会失败。 >针对:https://issues.apache.org/jira/browse/FLINK-16681,已经修复此问题,但是我们使用1.11最新版本,运行一段时间后,发现还是会有此问题,如何解决 > > > >-- >Sent from: http://apache-flink.147419.n8.nabble.com/ |
In reply to this post by yanzi
Hi,yanzi
可以贴下完整的报错信息吗? 祝好, Leonard Xu > 在 2020年12月3日,10:36,yanzi <[hidden email]> 写道: > > 使用flinksql 1.11版本,运行计算好的指标落入mysql,长时间没有数据会报错,导致任务会失败。 > 针对:https://issues.apache.org/jira/browse/FLINK-16681,已经修复此问题,但是我们使用1.11最新版本,运行一段时间后,发现还是会有此问题,如何解决 > > > > -- > Sent from: http://apache-flink.147419.n8.nabble.com/ |
hi Leonard:
报错信息如下: [2020-12-02 22:01:03.403] [ERROR] [jdbc-upsert-output-format-thread-1] [org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat] >>> JDBC executeBatch error, retry times = 3 java.sql.SQLException: No operations allowed after statement closed. at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.setString(PreparedStatement.java:3943) ~[mysql-connector-java-5.1.49.jar:5.1.49] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$createExternalConverter$57fde215$8(AbstractJdbcRowConverter.java:219) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$wrapIntoNullableExternalConverter$2bf50691$1(AbstractJdbcRowConverter.java:193) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.toExternal(AbstractJdbcRowConverter.java:86) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.lambda$createKeyedRowExecutor$3fd497bb$1(JdbcRowDataOutputFormat.java:164) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.executor.KeyedBatchStatementExecutor.executeBatch(KeyedBatchStatementExecutor.java:71) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:154) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_262] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] -- Sent from: http://apache-flink.147419.n8.nabble.com/ |
这个应该只是个 error 的日志,方便也发下retry times = 1 和 retry times = 2 的日志看下吗
在 2020-12-03 16:17:27,"yanzi" <[hidden email]> 写道: >hi Leonard: > >报错信息如下: >[2020-12-02 22:01:03.403] [ERROR] [jdbc-upsert-output-format-thread-1] >[org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat] >>> JDBC >executeBatch error, retry times = 3 >java.sql.SQLException: No operations allowed after statement closed. > at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at com.mysql.jdbc.PreparedStatement.setString(PreparedStatement.java:3943) >~[mysql-connector-java-5.1.49.jar:5.1.49] > at >org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$createExternalConverter$57fde215$8(AbstractJdbcRowConverter.java:219) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$wrapIntoNullableExternalConverter$2bf50691$1(AbstractJdbcRowConverter.java:193) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.toExternal(AbstractJdbcRowConverter.java:86) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.lambda$createKeyedRowExecutor$3fd497bb$1(JdbcRowDataOutputFormat.java:164) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.internal.executor.KeyedBatchStatementExecutor.executeBatch(KeyedBatchStatementExecutor.java:71) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:154) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at >org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) >~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) >[?:1.8.0_262] > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) >[?:1.8.0_262] > at >java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) >[?:1.8.0_262] > at >java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) >[?:1.8.0_262] > at >java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) >[?:1.8.0_262] > at >java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) >[?:1.8.0_262] > at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] > > > >-- >Sent from: http://apache-flink.147419.n8.nabble.com/ |
1、retry times =1报错日志:
[2020-12-02 22:01:00.800] [ERROR] [jdbc-upsert-output-format-thread-1] [org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat] >>> JDBC executeBatch error, retry times = 1 java.sql.SQLException: No operations allowed after statement closed. at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.clearBatch(PreparedStatement.java:1051) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.executeBatchInternal(PreparedStatement.java:1323) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.executeBatch(StatementImpl.java:954) ~[mysql-connector-java-5.1.49.jar:5.1.49] at org.apache.flink.connector.jdbc.internal.executor.SimpleBatchStatementExecutor.executeBatch(SimpleBatchStatementExecutor.java:71) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.attemptFlush(JdbcBatchingOutputFormat.java:200) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:153) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_262] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] 2、retry times =2: [2020-12-02 22:01:01.402] [ERROR] [jdbc-upsert-output-format-thread-1] [org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat] >>> JDBC executeBatch error, retry times = 2 java.sql.SQLException: No operations allowed after statement closed. at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.setString(PreparedStatement.java:3943) ~[mysql-connector-java-5.1.49.jar:5.1.49] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$createExternalConverter$57fde215$8(AbstractJdbcRowConverter.java:219) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$wrapIntoNullableExternalConverter$2bf50691$1(AbstractJdbcRowConverter.java:193) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.toExternal(AbstractJdbcRowConverter.java:86) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.lambda$createKeyedRowExecutor$3fd497bb$1(JdbcRowDataOutputFormat.java:164) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.executor.KeyedBatchStatementExecutor.executeBatch(KeyedBatchStatementExecutor.java:71) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:154) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_262] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] 3、retry times = 3报错日志: [2020-12-02 22:01:03.403] [ERROR] [jdbc-upsert-output-format-thread-1] [org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat] >>> JDBC executeBatch error, retry times = 3 java.sql.SQLException: No operations allowed after statement closed. at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.setString(PreparedStatement.java:3943) ~[mysql-connector-java-5.1.49.jar:5.1.49] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$createExternalConverter$57fde215$8(AbstractJdbcRowConverter.java:219) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$wrapIntoNullableExternalConverter$2bf50691$1(AbstractJdbcRowConverter.java:193) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.toExternal(AbstractJdbcRowConverter.java:86) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.lambda$createKeyedRowExecutor$3fd497bb$1(JdbcRowDataOutputFormat.java:164) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.executor.KeyedBatchStatementExecutor.executeBatch(KeyedBatchStatementExecutor.java:71) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:154) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_262] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] 4、最后的报错日志: [2020-12-02 22:01:10.312] [ERROR] [GlobalGroupAggregate(groupBy=[act_name, parent_cut_act_type, fst_category, $e], partialFinalType=[FINAL], select=[act_name, parent_cut_act_type, fst_category, $e, $SUM0_RETRACT(count$0) AS $f4, SUM_RETRACT((sum$1, count$2)) AS $f5, $SUM0_RETRACT(count$3) AS $f6]) -> Calc(select=[CAST(act_name) AS act_name, CAST(parent_cut_act_type) AS parent_cut_act_no, CAST((fst_category IS NOT NULL CASE CAST(fst_category) CASE _UTF-16LE'ALL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")) AS fst_category, saleP AS salep, CAST(spu_num) AS spu_num, CAST(ordern) AS ordern]) -> Sink: Sink(table=[hive.temp_vipflink.mysql_realtime_trd_order_coupon_index], fields=[act_name, parent_cut_act_no, fst_category, salep, spu_num, ordern]) (7/20)] [org.apache.flink.streaming.runtime.tasks.StreamTask] >>> Error during disposal of stream operator. java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.close(JdbcBatchingOutputFormat.java:234) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.close(JdbcRowDataOutputFormat.java:139) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction.close(OutputFormatSinkFunction.java:97) ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.api.common.functions.util.FunctionUtils.closeFunction(FunctionUtils.java:43) ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:117) ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:729) [flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUpInvoke(StreamTask.java:645) [flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:549) [flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) [flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) [flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_262] Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.lang.RuntimeException: Writing records to JDBC failed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.checkFlushException(JdbcBatchingOutputFormat.java:142) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:167) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.io.IOException: java.sql.SQLException: No operations allowed after statement closed. at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:177) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more Caused by: java.sql.SQLException: No operations allowed after statement closed. at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.StatementImpl.checkClosed(StatementImpl.java:426) ~[mysql-connector-java-5.1.49.jar:5.1.49] at com.mysql.jdbc.PreparedStatement.setString(PreparedStatement.java:3943) ~[mysql-connector-java-5.1.49.jar:5.1.49] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$createExternalConverter$57fde215$8(AbstractJdbcRowConverter.java:219) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.lambda$wrapIntoNullableExternalConverter$2bf50691$1(AbstractJdbcRowConverter.java:193) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.converter.AbstractJdbcRowConverter.toExternal(AbstractJdbcRowConverter.java:86) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.lambda$createKeyedRowExecutor$3fd497bb$1(JdbcRowDataOutputFormat.java:164) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.executor.KeyedBatchStatementExecutor.executeBatch(KeyedBatchStatementExecutor.java:71) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.table.JdbcRowDataOutputFormat.attemptFlush(JdbcRowDataOutputFormat.java:154) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.flush(JdbcBatchingOutputFormat.java:171) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat.lambda$open$0(JdbcBatchingOutputFormat.java:120) ~[flink-connector-jdbc_2.11-1.11.0.jar:1.11.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_262] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[?:1.8.0_262] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_262] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_262] ... 1 more -- Sent from: http://apache-flink.147419.n8.nabble.com/ |
Free forum by Nabble | Edit this page |