[ https://issues.apache.org/jira/browse/FLINK-35056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17875320#comment-17875320 ]
Sergei Morozov commented on FLINK-35056: ---------------------------------------- Here's a source table example that will fail the connector: {code:sql} CREATE TABLE dt_pk ( dt datetime NOT NULL PRIMARY KEY, val INT ); {code} The failure will look like: {noformat} org.apache.flink.util.FlinkRuntimeException: Generate Splits for table sergei.dbo.dt_src error at com.ververica.cdc.connectors.sqlserver.source.dialect.SqlServerChunkSplitter.generateSplits(SqlServerChunkSplitter.java:125) at com.ververica.cdc.connectors.base.source.assigner.SnapshotSplitAssigner.getNext(SnapshotSplitAssigner.java:188) at com.ververica.cdc.connectors.base.source.assigner.HybridSplitAssigner.getNext(HybridSplitAssigner.java:137) at com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator.assignSplits(IncrementalSourceEnumerator.java:174) at com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator.handleSplitRequest(IncrementalSourceEnumerator.java:97) at org.apache.flink.runtime.source.coordinator.SourceCoordinator.lambda$handleEventFromOperator$2(SourceCoordinator.java:230) at org.apache.flink.runtime.source.coordinator.SourceCoordinator.lambda$runInEventLoop$9(SourceCoordinator.java:406) at org.apache.flink.util.ThrowableCatchingRunnable.run(ThrowableCatchingRunnable.java:40) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:630) at java.base/java.lang.Thread.run(Thread.java:831) Caused by: org.apache.flink.table.api.ValidationException: Timestamp precision must be between 0 and 9 (both inclusive). at org.apache.flink.table.types.logical.TimestampType.<init>(TimestampType.java:85) at org.apache.flink.table.types.logical.TimestampType.<init>(TimestampType.java:97) at org.apache.flink.table.types.logical.TimestampType.<init>(TimestampType.java:101) at org.apache.flink.table.api.DataTypes.TIMESTAMP(DataTypes.java:406) at com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerTypeUtils.convertFromColumn(SqlServerTypeUtils.java:71) at com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerTypeUtils.fromDbzColumn(SqlServerTypeUtils.java:31) at com.ververica.cdc.connectors.sqlserver.source.dialect.SqlServerChunkSplitter.fromDbzColumn(SqlServerChunkSplitter.java:144) at com.ververica.cdc.connectors.base.source.assigner.splitter.JdbcSourceChunkSplitter.getSplitType(JdbcSourceChunkSplitter.java:144) at com.ververica.cdc.connectors.sqlserver.source.dialect.SqlServerChunkSplitter.generateSplits(SqlServerChunkSplitter.java:102) ... 13 common frames omitted {noformat} > when initial sqlserver table that's primary key is datetime type, it > org.apache.flink.table.api.ValidationException: Timestamp precision must be > between 0 and 9 (both inclusive) > ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- > > Key: FLINK-35056 > URL: https://issues.apache.org/jira/browse/FLINK-35056 > Project: Flink > Issue Type: Bug > Components: Flink CDC > Affects Versions: 2.0.0 > Reporter: fengfengzai > Priority: Critical > Attachments: sqlserver-bug.png > > > when initial sqlserver table that's primary key is datetime type. > it error: > org.apache.flink.table.api.ValidationException: Timestamp precision must be > between 0 and 9 (both inclusive) > > i find datetime's length is 23. it exceed 9. so it errors. -- This message was sent by Atlassian Jira (v8.20.10#820010)