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

Validate log position method missing gtid info from SourceInfo

      We expect to print the GTID information of SourceInfo when the previous connector offset is not available, it will hellp us to figure out the cause of connector failure.

      connector offset info:
      ["test_source_v5",

      {"server":"txy_testnet_cht_rate_new"}

      ]

      {"transaction_id":null,"ts_sec":1723528676,"file":"mysql-bin.003812","pos":182209085,"gtids":"a61802a0-8917-11ee-beef-08c0eb46d56e:1-719173120","row":1,"server_id":1677731605,"event":2}
      [2024-08-13 06:07:26,391] ERROR [test_source_v5|task-0] WorkerSourceTask{id=test_source_v5-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:191)
      io.debezium.DebeziumException: The connector is trying to read binlog starting at SourceInfo [currentGtid=null, currentBinlogFilename=mysql-bin.003812, currentBinlogPosition=182209085, currentRowNumber=0, serverId=0, sourceTime=null, threadId=-1, currentQuery=null, tableIds=[], databaseName=null], but this is no longer available on the server. Reconfigure the connector to use a snapshot when needed.
      	at io.debezium.connector.mysql.MySqlConnectorTask.validateSnapshotFeasibility(MySqlConnectorTask.java:365)
      	at io.debezium.connector.mysql.MySqlConnectorTask.start(MySqlConnectorTask.java:114)
      	at io.debezium.connector.common.BaseSourceTask.start(BaseSourceTask.java:130)
      	at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:232)
      	at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:189)
      	at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:238)
      	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.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
      	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
      	at java.base/java.lang.Thread.run(Thread.java:829)
      

            [DBZ-8140] Validate log position method missing gtid info from SourceInfo

            Re-opening and reverting via https://github.com/debezium/debezium/pull/5793

            The change in question introduced a regression in ReadOnlyIncrementalSnapshotIT - it has to be run against GTID enabled server.

            The test fails with

            java.util.concurrent.ExecutionException: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
            	at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
            	at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
            	at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:484)
            	at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:220)
            	at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27)
            	at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:432)
            	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)
            Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
            	at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
            	at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:579)
            	at io.debezium.connector.binlog.EventBuffer.consumeEvent(EventBuffer.java:191)
            	at io.debezium.connector.binlog.EventBuffer.add(EventBuffer.java:124)
            	at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$18(BinlogStreamingChangeEventSource.java:213)
            	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1268)
            	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1094)
            	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:653)
            	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:954)
            	... 1 common frames omitted
            Caused by: io.debezium.DebeziumException: Error processing binlog event
            	... 9 common frames omitted
            Caused by: java.lang.NumberFormatException: For input string: "1-11"
            	at java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:67)
            	at java.base/java.lang.Long.parseLong(Long.java:708)
            	at java.base/java.lang.Long.parseLong(Long.java:831)
            	at io.debezium.connector.mysql.MySqlReadOnlyIncrementalSnapshotContext.reachedHighWatermark(MySqlReadOnlyIncrementalSnapshotContext.java:88)
            	at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:161)
            	at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:79)
            	at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:35)
            	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:464)
            	at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleServerHeartbeat(BinlogStreamingChangeEventSource.java:621)
            	at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$1(BinlogStreamingChangeEventSource.java:175)
            	at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:559)
            	... 8 common frames omitted
            

            And the reason is that the code sets GTID in source info into and unexpected value.

            Jiri Pechanec added a comment - Re-opening and reverting via https://github.com/debezium/debezium/pull/5793 The change in question introduced a regression in ReadOnlyIncrementalSnapshotIT - it has to be run against GTID enabled server. The test fails with java.util.concurrent.ExecutionException: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped. at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122) at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191) at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:484) at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:220) at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27) at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:432) 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) Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped. at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:579) at io.debezium.connector.binlog.EventBuffer.consumeEvent(EventBuffer.java:191) at io.debezium.connector.binlog.EventBuffer.add(EventBuffer.java:124) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$18(BinlogStreamingChangeEventSource.java:213) at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1268) at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1094) at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:653) at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:954) ... 1 common frames omitted Caused by: io.debezium.DebeziumException: Error processing binlog event ... 9 common frames omitted Caused by: java.lang.NumberFormatException: For input string: "1-11" at java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:67) at java.base/java.lang.Long.parseLong(Long.java:708) at java.base/java.lang.Long.parseLong(Long.java:831) at io.debezium.connector.mysql.MySqlReadOnlyIncrementalSnapshotContext.reachedHighWatermark(MySqlReadOnlyIncrementalSnapshotContext.java:88) at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:161) at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:79) at io.debezium.connector.binlog.BinlogReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(BinlogReadOnlyIncrementalSnapshotChangeEventSource.java:35) at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:464) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleServerHeartbeat(BinlogStreamingChangeEventSource.java:621) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.lambda$execute$1(BinlogStreamingChangeEventSource.java:175) at io.debezium.connector.binlog.BinlogStreamingChangeEventSource.handleEvent(BinlogStreamingChangeEventSource.java:559) ... 8 common frames omitted And the reason is that the code sets GTID in source info into and unexpected value.

            Released

            Debezium Builder added a comment - Released

            Applied to main (=3.0).

            Chris Cranford added a comment - Applied to main (=3.0).

              ywyuewei Harvey Yue (Inactive)
              ywyuewei Harvey Yue (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

                Created:
                Updated: