[ https://issues.apache.org/jira/browse/FLINK-20552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17247002#comment-17247002 ]
Leonard Xu commented on FLINK-20552: ------------------------------------ Hi, [~meijies] Thanks for the report, A simple way to fix this is wrapping the `JdbcBatchingOutputFormat` with `GenericJdbcSinkFunction` [here|https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSink.java#L87%C2%A0] {code:java} return OutputFormatProvider.of(builder.build()); => return SinkFunctionProvider.of(new GenericJdbcSinkFunction<>(builder.build())); {code} And we can add a test for this change, [~meijies] Do you like to help fix this ? > Jdbc connector don't sink buffer data when checkpoint > ----------------------------------------------------- > > Key: FLINK-20552 > URL: https://issues.apache.org/jira/browse/FLINK-20552 > Project: Flink > Issue Type: Bug > Components: Connectors / JDBC, Table SQL / Ecosystem > Reporter: mei jie > Priority: Major > Labels: starter > > JdbcBatchingOutputFormat is wrapped to OutputFormatSinkFunction``` when > createSinkTransformation at CommonPhysicalSink class. but > OutputFormatSinkFunction don't implement CheckpointedFunction interface, so > the flush method of JdbcBatchingOutputFormat can't be called when checkpoint -- This message was sent by Atlassian Jira (v8.3.4#803005)