Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-9209

Incremental snapshot offset failing to load on task restart

XMLWordPrintable

      In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      SQL Server on v3.1.2

      What is the connector configuration

       

      { "connector.class":"io.debezium.connector.sqlserver.SqlServerConnector", "database.hostname":"x.x.x.x", "database.port":"1433", "database.user":"bgoyal_test_login", "database.password":"<PASSWORD>", "database.names":"bgoyal_test", "topic.prefix":"test2", "tasks.max":"1", "schema.history.internal.kafka.bootstrap.servers":"localhost:9092", "schema.history.internal.kafka.topic":"history2", "table.include.list":"dbo.customers, dbo.dbz_signaling, dbo.MET Romania\\$Taget", "signal.data.collection":"bgoyal_test.dbo.dbz_signaling", "key.converter":"io.confluent.connect.avro.AvroConverter", "value.converter":"io.confluent.connect.avro.AvroConverter", "key.converter.schema.registry.url":"http://localhost:8081", "value.converter.schema.registry.url":"http://localhost:8081", "confluent.topic.bootstrap.servers":"localhost:9092", "incremental.snapshot.chunk.size":"1", "database.encrypt":"false", "schema.history.internal.store.only.captured.tables.ddl":true }

       

      What is the captured database version and mode of deployment?

      on-prem

      I am using a table which has spaces in name MET Romania$Taget . The connector is configured to capture changes of this table. A record is inserted into the DBZ signaling table to capture incremental snapshot of this table

      The connector successfully starts capturing incremental snapshot. If the task is restarted mid-way (the offset must have incremental_snapshot_collections_id to repro this issue), the connection fails to come up. 

      What behavior do you expect?

      No task failure and a seamless restart.

      What behavior do you see?

      The task fails on restart.

      Do you see the same behaviour using the latest released Debezium version?

      (Ideally, also verify with latest Alpha/Beta/CR version)

      <Your answer>

      Do you have the connector logs, ideally from start till finish?

      (You might be asked later to provide DEBUG/TRACE level log)

      ```

       

      [2025-07-08 16:13:23,652] ERROR [test_v3_2|task-0] WorkerSourceTask{id=test_v3_2-0} Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask:237)

      java.lang.IllegalArgumentException: Unexpected input: R

      at io.debezium.relational.TableIdParser$ParsingState$3.handleCharacter(TableIdParser.java:154)

      at io.debezium.relational.TableIdParser$TableIdTokenizer.tokenize(TableIdParser.java:71)

      at io.debezium.text.TokenStream.start(TokenStream.java:446)

      at io.debezium.relational.TableIdParser.parse(TableIdParser.java:36)

      at io.debezium.relational.TableIdParser.parse(TableIdParser.java:30)

      at io.debezium.relational.TableId.parseParts(TableId.java:80)

      at io.debezium.relational.TableId.parse(TableId.java:54)

      at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext$SnapshotDataCollection.lambda$stringToDataCollections$1(AbstractIncrementalSnapshotContext.java:469)

      at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)

      at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1708)

      at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)

      at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)

      at java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)

      at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)

      at java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:682)

      at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext$SnapshotDataCollection.stringToDataCollections(AbstractIncrementalSnapshotContext.java:472)

      at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext.init(AbstractIncrementalSnapshotContext.java:281)

      at io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext.load(SignalBasedIncrementalSnapshotContext.java:35)

      at io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext.load(SignalBasedIncrementalSnapshotContext.java:30)

      at io.debezium.connector.sqlserver.SqlServerOffsetContext$Loader.load(SqlServerOffsetContext.java:129)

      at io.debezium.connector.sqlserver.SqlServerOffsetContext$Loader.load(SqlServerOffsetContext.java:107)

      at io.debezium.connector.common.OffsetReader.lambda$offsets$0(OffsetReader.java:57)

      at java.base/java.lang.Iterable.forEach(Iterable.java:75)

      at io.debezium.connector.common.OffsetReader.offsets(OffsetReader.java:49)

      at io.debezium.connector.common.BaseSourceTask.getPreviousOffsets(BaseSourceTask.java:520)

      at io.debezium.connector.sqlserver.SqlServerConnectorTask.start(SqlServerConnectorTask.java:92)

      at io.debezium.connector.common.BaseSourceTask.start(BaseSourceTask.java:256)

      at org.apache.kafka.connect.runtime.AbstractWorkerSourceTask.initializeAndStart(AbstractWorkerSourceTask.java:283)

      at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:227)

      at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:284)

      at org.apache.kafka.connect.runtime.AbstractWorkerSourceTask.run(AbstractWorkerSourceTask.java:80)

      at org.apache.kafka.connect.runtime.isolation.Plugins.lambda$withClassLoader$7(Plugins.java:339)

      at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)

      at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)

      at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)

      at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)

      at java.base/java.lang.Thread.run(Thread.java:1583)

      [2025-07-08 16:13:23,652] INFO [test_v3_2|task-0] Stopping down connector (io.debezium.connector.common.BaseSourceTask:437)

      [2025-07-08 16:13:23,652] INFO [test_v3_2|task-0] [Producer clientId=test2-schemahistory] Closing the Kafka producer with timeoutMillis = 30000 ms. (org.apache.kafka.clients.producer.KafkaProducer:1347)

      ```

      How to reproduce the issue using our tutorial deployment?

      <Your answer>

      Feature request or enhancement

      For feature requests or enhancements, provide this information, please:

      Which use case/requirement will be addressed by the proposed feature?

      zulip chat: #community-sqlserver > Incremental snapshot offset failing to load on task restart

      Implementation ideas (optional)

      <Your answer>

              Unassigned Unassigned
              bhagyashreegoyal Bhagyashree Goyal
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

                Created:
                Updated:
                Resolved: