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

ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal fails randomly

    XMLWordPrintable

Details

    • False
    • None
    • False

    Description

      ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal fail randomly (e.g. here) with

      2024-02-21T17:24:41.9625665Z 2024-02-21 17:24:41,850 ERROR  MySQL|is_test||binlog|  Error during binlog processing. Last offset stored = {ts_sec=1708536233, file=mysql-bin.000002, pos=12520521, incremental_snapshot_signal_offset=null, gtids=7f45695d-d0d
      b-11ee-8282-0242ac110002:1-15988, server_id=112233, event=1}, binlog reader near position = mysql-bin.000002/12520521   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:24:41.9629384Z 2024-02-21 17:24:41,850 ERROR  MySQL|is_test||binlog|  Producer failure   [io.debezium.pipeline.ErrorHandler]
      2024-02-21T17:24:41.9630961Z io.debezium.DebeziumException: Error processing binlog event
      2024-02-21T17:24:41.9632986Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-21T17:24:41.9634644Z    at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-21T17:24:41.9635854Z    at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-21T17:24:41.9637532Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-21T17:24:41.9639449Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-21T17:24:41.9641008Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-21T17:24:41.9642437Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-21T17:24:41.9643664Z    at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-21T17:24:41.9644809Z    at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-21T17:24:41.9646099Z Caused by: io.debezium.DebeziumException: Database error while executing incremental snapshot for table 'null'
      2024-02-21T17:24:41.9648185Z    at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:418)
      2024-02-21T17:24:41.9651045Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:130)
      2024-02-21T17:24:41.9653985Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-21T17:24:41.9656894Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-21T17:24:41.9659074Z    at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-21T17:24:41.9660879Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-21T17:24:41.9662917Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-21T17:24:41.9664735Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-21T17:24:41.9665901Z    ... 8 common frames omitted
      2024-02-21T17:24:41.9668162Z Caused by: java.lang.NullPointerException: Cannot invoke "io.debezium.pipeline.source.snapshot.incremental.DataCollection.getId()" because the return value of "io.debezium.pipeline.source.snapshot.incremental.IncrementalSnap
      shotContext.currentDataCollectionId()" is null
      2024-02-21T17:24:41.9671387Z    at io.debezium.pipeline.notification.IncrementalSnapshotNotificationService.notifyInProgress(IncrementalSnapshotNotificationService.java:132)
      2024-02-21T17:24:41.9674081Z    at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:405)
      2024-02-21T17:24:41.9675741Z    ... 15 common frames omitted
      2024-02-21T17:24:41.9678277Z 2024-02-21 17:24:41,851 INFO   MySQL|is_test||binlog|  Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored.   [io
      .debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:24:41.9681439Z 2024-02-21 17:24:41,855 ERROR  ReadOnlyIncrementalSnapshotIT|||engine|  Engine has failed with    [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:24:41.9683707Z java.util.concurrent.ExecutionException: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-21T17:24:41.9685855Z    at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
      2024-02-21T17:24:41.9686923Z    at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
      2024-02-21T17:24:41.9688153Z    at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:454)
      2024-02-21T17:24:41.9689668Z    at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:204)
      2024-02-21T17:24:41.9691007Z    at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27)
      2024-02-21T17:24:41.9692489Z    at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:423)
      2024-02-21T17:24:41.9693837Z    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      2024-02-21T17:24:41.9695151Z    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      2024-02-21T17:24:41.9696257Z    at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-21T17:24:41.9697611Z Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-21T17:24:41.9699198Z    at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
      2024-02-21T17:24:41.9700697Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-21T17:24:41.9702157Z    at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-21T17:24:41.9703350Z    at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-21T17:24:41.9704779Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-21T17:24:41.9706519Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-21T17:24:41.9708191Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-21T17:24:41.9709759Z    at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-21T17:24:41.9711162Z    at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-21T17:24:41.9712372Z    ... 1 common frames omitted
      2024-02-21T17:24:41.9713156Z Caused by: io.debezium.DebeziumException: Error processing binlog event
      2024-02-21T17:24:41.9714004Z    ... 9 common frames omitted
      2024-02-21T17:24:41.9715300Z Caused by: io.debezium.DebeziumException: Database error while executing incremental snapshot for table 'null'
      2024-02-21T17:24:41.9717765Z    at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:418)
      2024-02-21T17:24:41.9721085Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:130)
      2024-02-21T17:24:41.9724552Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-21T17:24:41.9727799Z    at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-21T17:24:41.9730301Z    at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-21T17:24:41.9732332Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-21T17:24:41.9734606Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-21T17:24:41.9736705Z    at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-21T17:24:41.9738034Z    ... 8 common frames omitted
      2024-02-21T17:24:41.9740730Z Caused by: java.lang.NullPointerException: Cannot invoke "io.debezium.pipeline.source.snapshot.incremental.DataCollection.getId()" because the return value of "io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext.currentDataCollectionId()" is null
      2024-02-21T17:24:41.9744491Z    at io.debezium.pipeline.notification.IncrementalSnapshotNotificationService.notifyInProgress(IncrementalSnapshotNotificationService.java:132)
      2024-02-21T17:24:41.9747628Z    at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:405)
      2024-02-21T17:24:41.9749755Z    ... 15 common frames omitted
      

      Full test log:

      2024-02-21T17:23:51.6224883Z 2024-02-21 17:23:51,527 INFO   NotificationsIT|||test|  Starting test io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:23:51.6228146Z 2024-02-21 17:23:51,528 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:23:51.6582637Z 17:23:51.657 mysql-gtids2024-02-21T17:23:51.657405Z 1703 [Warning] [MY-013360] [Server] Plugin mysql_native_password reported: ''mysql_native_password' is deprecated and will be removed in a future release. Please use caching_sha2_password instead'
      2024-02-21T17:23:51.7221527Z 2024-02-21 17:23:51,632 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-21T17:23:52.0700971Z 17:23:52.069 mysql-gtids2024-02-21T17:23:52.068971Z 1704 [Warning] [MY-013360] [Server] Plugin mysql_native_password reported: ''mysql_native_password' is deprecated and will be removed in a future release. Please use caching_sha2_password instead'
      2024-02-21T17:23:52.0784681Z 17:23:52.077 mysql-gtids2024-02-21T17:23:52.077904Z 1705 [Warning] [MY-013360] [Server] Plugin mysql_native_password reported: ''mysql_native_password' is deprecated and will be removed in a future release. Please use caching_sha2_password instead'
      2024-02-21T17:23:52.0906272Z 17:23:52.090 mysql-gtids2024-02-21T17:23:52.089940Z 1706 [Warning] [MY-013360] [Server] Plugin mysql_native_password reported: ''mysql_native_password' is deprecated and will be removed in a future release. Please use caching_sha2_password instead'
      2024-02-21T17:23:52.1225088Z 2024-02-21 17:23:52,064 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-21T17:23:52.1227543Z 2024-02-21 17:23:52,066 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'CREATING' to 'INITIALIZING'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:23:52.1230920Z 2024-02-21 17:23:52,066 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Loading the custom source info struct maker plugin: io.debezium.connector.mysql.MySqlSourceInfoStructMaker   [io.debezium.config.CommonConnectorConfig]
      2024-02-21T17:23:52.1234253Z 2024-02-21 17:23:52,067 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Using io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter   [io.debezium.connector.mysql.MySqlConnectorConfig]
      2024-02-21T17:23:52.1237142Z 2024-02-21 17:23:52,070 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Successfully tested connection for jdbc:mysql://localhost:4302/?useInformationSchema=true&nullCatalogMeansCurrent=false&useUnicode=true&characterEncoding=UTF-8&characterSetResults=UTF-8&zeroDateTimeBehavior=CONVERT_TO_NULL&connectTimeout=30000 with user 'mysqluser'   [io.debezium.connector.mysql.MySqlConnector]
      2024-02-21T17:23:52.1239359Z 2024-02-21 17:23:52,070 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-21T17:23:52.1240691Z 2024-02-21 17:23:52,071 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'INITIALIZING' to 'CREATING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:23:52.1242591Z 2024-02-21 17:23:52,071 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'CREATING_TASKS' to 'STARTING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:23:52.1244221Z 2024-02-21 17:23:52,072 INFO   ||||  Starting MySqlConnectorTask with configuration:   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1246397Z 2024-02-21 17:23:52,072 INFO   ||||     connector.class = io.debezium.connector.mysql.MySqlConnector   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1247797Z 2024-02-21 17:23:52,072 INFO   ||||     message.key.columns = incremental_snapshot-test_psmsse.a42:pk1,pk2,pk3,pk4   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1249260Z 2024-02-21 17:23:52,072 INFO   ||||     incremental.snapshot.chunk.size = 1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1250538Z 2024-02-21 17:23:52,072 INFO   ||||     record.processing.shutdown.timeout.ms = 1000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1251648Z 2024-02-21 17:23:52,072 INFO   ||||     signal.enabled.channels = source,kafka   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1252799Z 2024-02-21 17:23:52,072 INFO   ||||     include.schema.changes = false   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1254226Z 2024-02-21 17:23:52,072 INFO   ||||     record.processing.order = ORDERED   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1255231Z 2024-02-21 17:23:52,072 INFO   ||||     connector.adapter = mysql   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1257115Z 2024-02-21 17:23:52,072 INFO   ||||     schema.history.internal.file.filename = /home/runner/work/debezium/debezium/debezium-connector-mysql/target/data/file-schema-history-is.txt   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1258753Z 2024-02-21 17:23:52,072 INFO   ||||     topic.prefix = is_test   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1260781Z 2024-02-21 17:23:52,072 INFO   ||||     database.jdbc.driver = com.mysql.cj.jdbc.Driver   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1262846Z 2024-02-21 17:23:52,072 INFO   ||||     offset.storage.file.filename = /home/runner/work/debezium/debezium/debezium-connector-mysql/target/data/file-connector-offsets.txt   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1264290Z 2024-02-21 17:23:52,072 INFO   ||||     poll.interval.ms = 10   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1265453Z 2024-02-21 17:23:52,072 INFO   ||||     signal.data.collection = incremental_snapshot-test_psmsse.debezium_signal   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1266962Z 2024-02-21 17:23:52,072 INFO   ||||     record.processing.threads = 4   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1268116Z 2024-02-21 17:23:52,072 INFO   ||||     errors.retry.delay.initial.ms = 300   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1269233Z 2024-02-21 17:23:52,072 INFO   ||||     binlog.buffer.size = 10000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1270504Z 2024-02-21 17:23:52,072 INFO   ||||     value.converter = org.apache.kafka.connect.json.JsonConverter   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1272027Z 2024-02-21 17:23:52,072 INFO   ||||     key.converter = org.apache.kafka.connect.json.JsonConverter   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1273164Z 2024-02-21 17:23:52,072 INFO   ||||     database.user = mysqluser   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1274246Z 2024-02-21 17:23:52,072 INFO   ||||     signal.kafka.bootstrap.servers = localhost:36149   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1275333Z 2024-02-21 17:23:52,072 INFO   ||||     database.server.id = 18765   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1276565Z 2024-02-21 17:23:52,072 INFO   ||||     offset.storage = org.apache.kafka.connect.storage.FileOffsetBackingStore   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1278285Z 2024-02-21 17:23:52,072 INFO   ||||     signal.kafka.topic = incremental_snapshot-test_psmssesignals_topic   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1279560Z 2024-02-21 17:23:52,072 INFO   ||||     signal.poll.interval.ms = 1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1280600Z 2024-02-21 17:23:52,072 INFO   ||||     read.only = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1281681Z 2024-02-21 17:23:52,072 INFO   ||||     offset.flush.timeout.ms = 5000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1282700Z 2024-02-21 17:23:52,072 INFO   ||||     errors.retry.delay.max.ms = 10000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1283938Z 2024-02-21 17:23:52,072 INFO   ||||     incremental.snapshot.allow.schema.changes = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1284982Z 2024-02-21 17:23:52,072 INFO   ||||     database.port = 4302   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1286153Z 2024-02-21 17:23:52,076 INFO   ||||     offset.flush.interval.ms = 0   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1287174Z 2024-02-21 17:23:52,076 INFO   ||||     database.ssl.mode = disabled   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1288425Z 2024-02-21 17:23:52,076 INFO   ||||     internal.task.management.timeout.ms = 180000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1289504Z 2024-02-21 17:23:52,076 INFO   ||||     database.protocol = jdbc:mysql   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1290852Z 2024-02-21 17:23:52,076 INFO   ||||     schema.history.internal = io.debezium.storage.file.history.FileSchemaHistory   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1292279Z 2024-02-21 17:23:52,076 INFO   ||||     record.processing.with.serial.consumer = false   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1293476Z 2024-02-21 17:23:52,076 INFO   ||||     table.exclude.list = incremental_snapshot-test_psmsse.b   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1294548Z 2024-02-21 17:23:52,076 INFO   ||||     errors.max.retries = -1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1295529Z 2024-02-21 17:23:52,076 INFO   ||||     database.hostname = localhost   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1296514Z 2024-02-21 17:23:52,076 INFO   ||||     database.password = ********   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1297549Z 2024-02-21 17:23:52,076 INFO   ||||     schema.name.adjustment.mode = avro   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1298684Z 2024-02-21 17:23:52,076 INFO   ||||     name = testing-connector   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1299650Z 2024-02-21 17:23:52,076 INFO   ||||     include.query = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1300708Z 2024-02-21 17:23:52,076 INFO   ||||     snapshot.mode = schema_only   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1301988Z 2024-02-21 17:23:52,076 INFO   ||||     database.include.list = incremental_snapshot-test_psmsse   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1304076Z 2024-02-21 17:23:52,076 INFO   ||||  Loading the custom source info struct maker plugin: io.debezium.connector.mysql.MySqlSourceInfoStructMaker   [io.debezium.config.CommonConnectorConfig]
      2024-02-21T17:23:52.1305833Z 2024-02-21 17:23:52,076 INFO   ||||  Using io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter   [io.debezium.connector.mysql.MySqlConnectorConfig]
      2024-02-21T17:23:52.1308364Z 2024-02-21 17:23:52,076 INFO   ||||  Loading the custom topic naming strategy plugin: io.debezium.schema.DefaultTopicNamingStrategy   [io.debezium.config.CommonConnectorConfig]
      2024-02-21T17:23:52.1317736Z 2024-02-21 17:23:52,077 INFO   ||||  No previous offsets found   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:23:52.1319806Z 2024-02-21 17:23:52,088 INFO   ||||  Closing connection before starting schema recovery   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-21T17:23:52.1321547Z 2024-02-21 17:23:52,089 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-21T17:23:52.1327847Z 2024-02-21 17:23:52,089 INFO   ||||  Connector started for the first time, database schema history recovery will not be executed   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-21T17:23:52.1331423Z 2024-02-21 17:23:52,089 INFO   ||||  Reconnecting after finishing schema recovery   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-21T17:23:52.1333550Z 2024-02-21 17:23:52,090 INFO   ||||  No previous offset found   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-21T17:23:52.2226739Z 2024-02-21 17:23:52,091 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = SignalProcessor   [io.debezium.util.Threads]
      2024-02-21T17:23:52.2229732Z 2024-02-21 17:23:52,128 INFO   ||||  Subscribing to signals topic 'incremental_snapshot-test_psmssesignals_topic'   [io.debezium.pipeline.signal.channels.KafkaSignalChannel]
      2024-02-21T17:23:52.2233014Z 2024-02-21 17:23:52,130 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = change-event-source-coordinator   [io.debezium.util.Threads]
      2024-02-21T17:23:52.2235950Z 2024-02-21 17:23:52,130 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = blocking-snapshot   [io.debezium.util.Threads]
      2024-02-21T17:23:52.2238630Z 2024-02-21 17:23:52,130 INFO   ||||  Creating thread debezium-mysqlconnector-is_test-change-event-source-coordinator   [io.debezium.util.Threads]
      2024-02-21T17:23:52.2241740Z 2024-02-21 17:23:52,130 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'STARTING_TASKS' to 'POLLING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:23:52.2245486Z 2024-02-21 17:23:52,130 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Using io.debezium.embedded.async.ParallelSmtConsumerProcessor processor   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:23:52.3227952Z 2024-02-21 17:23:52,266 INFO   MySQL|is_test|0|snapshot|  Read binlog position of MySQL primary server   [io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter]
      2024-02-21T17:23:52.3232083Z 2024-02-21 17:23:52,266 INFO   MySQL|is_test|0|snapshot|  	 using binlog 'mysql-bin.000002' at position '12520521' and gtid '7f45695d-d0db-11ee-8282-0242ac110002:1-15988'   [io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter]
      2024-02-21T17:23:53.7871366Z 17:23:53.786 mysql-gtids2024-02-21T17:23:53.786244Z 1707 [Warning] [MY-013360] [Server] Plugin mysql_native_password reported: ''mysql_native_password' is deprecated and will be removed in a future release. Please use caching_sha2_password instead'
      2024-02-21T17:23:53.8243030Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8245762Z Feb 21, 2024 5:23:53 PM com.github.shyiko.mysql.binlog.BinaryLogClient connect
      2024-02-21T17:23:53.8247301Z INFO: Connected to localhost:4302 at 7f45695d-d0db-11ee-8282-0242ac110002:1-15988 (sid:18765, cid:1707)
      2024-02-21T17:23:53.8248724Z Feb 21, 2024 5:23:53 PM com.github.shyiko.mysql.binlog.BinaryLogClient notifyEventListeners
      2024-02-21T17:23:53.8252917Z WARNING: io.debezium.connector.mysql.MySqlStreamingChangeEventSource$$Lambda$677/0x00007f8bf051b220@273c6865 choked on Event{header=EventHeaderV4{timestamp=1708535172000, eventType=FORMAT_DESCRIPTION, serverId=112233, headerLength=19, dataLength=103, nextPosition=126, flags=0}, data=FormatDescriptionEventData{binlogVersion=4, serverVersion='8.2.0', headerLength=19, dataLength=98, checksumType=CRC32}}
      2024-02-21T17:23:53.8256514Z java.lang.NullPointerException
      2024-02-21T17:23:53.8256906Z 
      2024-02-21T17:23:53.8257522Z Feb 21, 2024 5:23:53 PM com.github.shyiko.mysql.binlog.BinaryLogClient notifyEventListeners
      2024-02-21T17:23:53.8261251Z WARNING: io.debezium.connector.mysql.MySqlStreamingChangeEventSource$$Lambda$677/0x00007f8bf051b220@273c6865 choked on Event{header=EventHeaderV4{timestamp=1708535172000, eventType=PREVIOUS_GTIDS, serverId=112233, headerLength=19, dataLength=12, nextPosition=157, flags=128}, data=PreviousGtidSetEventData {gtidSet=''}}
      2024-02-21T17:23:53.8264046Z java.lang.NullPointerException
      2024-02-21T17:23:53.8264549Z 
      2024-02-21T17:23:53.8267241Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8272227Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8277579Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a4.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a4.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8282354Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a4.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a4.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8287137Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a4.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a4.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8291912Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a42.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a42.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8296587Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a42.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a42.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8301511Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a42.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a42.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8306447Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_date.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_date.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8315212Z 2024-02-21 17:23:53,771 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_date.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_date.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8320157Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_date.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_date.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8324858Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_dt.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_dt.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8329570Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_dt.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_dt.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8334191Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.a_dt.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.a_dt.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8338673Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.c.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.c.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8343134Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.c.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.c.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8347534Z 2024-02-21 17:23:53,772 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.c.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.c.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8352393Z 2024-02-21 17:23:53,773 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.debezium_signal.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.debezium_signal.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8357231Z 2024-02-21 17:23:53,773 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.debezium_signal.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.debezium_signal.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8362146Z 2024-02-21 17:23:53,773 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_psmsse.debezium_signal.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_psmsse.debezium_signal.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-21T17:23:53.8365841Z 2024-02-21 17:23:53,774 INFO   MySQL|is_test|0|snapshot|  Snapshot - Final stage   [io.debezium.pipeline.source.AbstractSnapshotChangeEventSource]
      2024-02-21T17:23:53.8368717Z 2024-02-21 17:23:53,774 INFO   MySQL|is_test|0|snapshot|  Snapshot completed   [io.debezium.pipeline.source.AbstractSnapshotChangeEventSource]
      2024-02-21T17:23:53.8371379Z 2024-02-21 17:23:53,774 INFO   MySQL|is_test|0|streaming|  Requested thread factory for connector MySqlConnector, id = is_test named = binlog-client   [io.debezium.util.Threads]
      2024-02-21T17:23:53.8374713Z 2024-02-21 17:23:53,783 INFO   MySQL|is_test|0|streaming|  No incremental snapshot in progress, no action needed on start   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-21T17:23:53.8377822Z 2024-02-21 17:23:53,784 INFO   MySQL|is_test|0|streaming|  SignalProcessor started. Scheduling it every 1ms   [io.debezium.pipeline.signal.SignalProcessor]
      2024-02-21T17:23:53.8380857Z 2024-02-21 17:23:53,784 INFO   MySQL|is_test|0|streaming|  Creating thread debezium-mysqlconnector-is_test-SignalProcessor   [io.debezium.util.Threads]
      2024-02-21T17:23:53.8383438Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  GTID set purged on server: ''   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8386079Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Attempting to generate a filtered GTID set   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8389150Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  GTID set from previous recorded offset: 7f45695d-d0db-11ee-8282-0242ac110002:1-15988   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8392677Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  GTID set available on server: 7f45695d-d0db-11ee-8282-0242ac110002:1-15988   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8395669Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Using first available positions for new GTID channels   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8398980Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Relevant GTID set available on server: 7f45695d-d0db-11ee-8282-0242ac110002:1-15988   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8402461Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Final merged GTID set to use when connecting to MySQL: 7f45695d-d0db-11ee-8282-0242ac110002:1-15988   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-21T17:23:53.8405927Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Registering binlog reader with GTID set: '7f45695d-d0db-11ee-8282-0242ac110002:1-15988'   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8408840Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Skip 0 events on streaming start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8411307Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Skip 0 rows on streaming start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8433700Z 2024-02-21 17:23:53,785 INFO   MySQL|is_test|0|streaming|  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-21T17:23:53.8436067Z 2024-02-21 17:23:53,786 INFO   ||||  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-21T17:23:53.8446663Z 2024-02-21 17:23:53,791 INFO   MySQL|is_test||binlog|  Connected to binlog at localhost:4302, starting at MySqlOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.mysql.Source:STRUCT}, sourceInfo=SourceInfo [currentGtid=null, currentBinlogFilename=mysql-bin.000002, currentBinlogPosition=12520521, currentRowNumber=0, serverId=0, sourceTime=2024-02-21T17:23:53.773Z, threadId=-1, currentQuery=null, tableIds=[incremental_snapshot-test_psmsse.debezium_signal], databaseName=year_test_1ua39rh], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=7f45695d-d0db-11ee-8282-0242ac110002:1-15988, currentGtidSet=7f45695d-d0db-11ee-8282-0242ac110002:1-15988, restartBinlogFilename=mysql-bin.000002, restartBinlogPosition=12520521, restartRowsToSkip=0, restartEventsToSkip=0, currentEventLengthInBytes=0, inTransaction=false, transactionId=null, incrementalSnapshotContext =IncrementalSnapshotContext [windowOpened=false, chunkEndPosition=null, dataCollectionsToSnapshot=[], lastEventKeySent=null, maximumKey=null]]   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8457526Z 2024-02-21 17:23:53,792 INFO   MySQL|is_test|0|streaming|  Waiting for keepalive thread to start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:53.8460388Z 2024-02-21 17:23:53,792 INFO   MySQL|is_test||binlog|  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-21T17:23:53.9256265Z 2024-02-21 17:23:53,892 INFO   MySQL|is_test|0|streaming|  Keepalive thread is running   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:23:58.3289877Z 2024-02-21 17:23:58,292 INFO   ||||  Requested 'INCREMENTAL' snapshot of data collections '[incremental_snapshot-test_psmsse.a]' with additional conditions '[]' and surrogate key 'PK of table will be used'   [io.debezium.pipeline.signal.actions.snapshotting.ExecuteSnapshot]
      2024-02-21T17:24:41.9576383Z 2024-02-21 17:24:41,820 INFO   MySQL|is_test||binlog|  Incremental snapshot's schema verification passed = true, schema = columns: {
      2024-02-21T17:24:41.9579953Z   pk INT(10, 0) NOT NULL
      2024-02-21T17:24:41.9580457Z   aa INT(10, 0) DEFAULT VALUE NULL
      2024-02-21T17:24:41.9580976Z }
      2024-02-21T17:24:41.9581329Z primary key: []
      2024-02-21T17:24:41.9581781Z default charset: null
      2024-02-21T17:24:41.9582198Z comment: null
      2024-02-21T17:24:41.9582584Z attributes: {
      2024-02-21T17:24:41.9583082Z }
      2024-02-21T17:24:41.9583981Z    [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext]
      2024-02-21T17:24:41.9587401Z 2024-02-21 17:24:41,821 INFO   MySQL|is_test||binlog|  Incremental snapshot for table 'incremental_snapshot-test_psmsse.a' will end at position [1000]   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-21T17:24:41.9599305Z 2024-02-21 17:24:41,832 INFO   ||||  6 records sent during previous 00:00:49.76, last recorded offset of {server=is_test} partition is {incremental_snapshot_correlation_id=null, ts_sec=1708536281, file=mysql-bin.000002, pos=12520521, incremental_snapshot_signal_offset=0, incremental_snapshot_maximum_key=aced0005757200135b4c6a6176612e6c616e672e4f626a6563743b90ce589f1073296c020000787000000001737200116a6176612e6c616e672e496e746567657212e2a0a4f781873802000149000576616c7565787200106a6176612e6c616e672e4e756d62657286ac951d0b94e08b0200007870000003e8, gtids=7f45695d-d0db-11ee-8282-0242ac110002:1-15988, server_id=112233, incremental_snapshot_collections=[{"incremental_snapshot_collections_id":"incremental_snapshot-test_psmsse.a","incremental_snapshot_collections_additional_condition":null,"incremental_snapshot_collections_surrogate_key":null}], incremental_snapshot_primary_key=aced0005757200135b4c6a6176612e6c616e672e4f626a6563743b90ce589f1073296c020000787000000001737200116a6176612e6c616e672e496e746567657212e2a0a4f781873802000149000576616c7565787200106a6176612e6c616e672e4e756d62657286ac951d0b94e08b020000787000000006}   [io.debezium.connector.common.BaseSourceTask]
      2024-02-21T17:24:41.9610061Z 2024-02-21 17:24:41,849 INFO   ||||  Requested stop of snapshot 'INCREMENTAL' for data collections '[incremental_snapshot-test_psmsse.a]'   [io.debezium.pipeline.signal.actions.snapshotting.StopSnapshot]
      2024-02-21T17:24:41.9613495Z 2024-02-21 17:24:41,850 INFO   ||||  Removing '[incremental_snapshot-test_psmsse.a]' collections from incremental snapshot   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-21T17:24:41.9616999Z 2024-02-21 17:24:41,850 INFO   ||||  Removed 'incremental_snapshot-test_psmsse.a' from incremental snapshot collection list.   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-21T17:24:41.9620387Z 2024-02-21 17:24:41,850 INFO   ||||  Advancing to next available collection in the incremental snapshot.   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-21T17:24:41.9625681Z 2024-02-21 17:24:41,850 ERROR  MySQL|is_test||binlog|  Error during binlog processing. Last offset stored = {ts_sec=1708536233, file=mysql-bin.000002, pos=12520521, incremental_snapshot_signal_offset=null, gtids=7f45695d-d0db-11ee-8282-0242ac110002:1-15988, server_id=112233, event=1}, binlog reader near position = mysql-bin.000002/12520521   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:24:41.9629399Z 2024-02-21 17:24:41,850 ERROR  MySQL|is_test||binlog|  Producer failure   [io.debezium.pipeline.ErrorHandler]
      2024-02-21T17:24:41.9631307Z io.debezium.DebeziumException: Error processing binlog event
      2024-02-21T17:24:41.9632994Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-21T17:24:41.9634652Z 	at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-21T17:24:41.9636032Z 	at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-21T17:24:41.9637539Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-21T17:24:41.9639456Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-21T17:24:41.9641013Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-21T17:24:41.9642444Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-21T17:24:41.9643676Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-21T17:24:41.9644815Z 	at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-21T17:24:41.9646105Z Caused by: io.debezium.DebeziumException: Database error while executing incremental snapshot for table 'null'
      2024-02-21T17:24:41.9648198Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:418)
      2024-02-21T17:24:41.9651051Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:130)
      2024-02-21T17:24:41.9653991Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-21T17:24:41.9656903Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-21T17:24:41.9659079Z 	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-21T17:24:41.9660888Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-21T17:24:41.9662932Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-21T17:24:41.9664741Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-21T17:24:41.9665906Z 	... 8 common frames omitted
      2024-02-21T17:24:41.9668169Z Caused by: java.lang.NullPointerException: Cannot invoke "io.debezium.pipeline.source.snapshot.incremental.DataCollection.getId()" because the return value of "io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext.currentDataCollectionId()" is null
      2024-02-21T17:24:41.9671396Z 	at io.debezium.pipeline.notification.IncrementalSnapshotNotificationService.notifyInProgress(IncrementalSnapshotNotificationService.java:132)
      2024-02-21T17:24:41.9674087Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:405)
      2024-02-21T17:24:41.9675754Z 	... 15 common frames omitted
      2024-02-21T17:24:41.9678289Z 2024-02-21 17:24:41,851 INFO   MySQL|is_test||binlog|  Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored.   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-21T17:24:41.9681446Z 2024-02-21 17:24:41,855 ERROR  ReadOnlyIncrementalSnapshotIT|||engine|  Engine has failed with    [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:24:41.9683889Z java.util.concurrent.ExecutionException: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-21T17:24:41.9685866Z 	at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
      2024-02-21T17:24:41.9686929Z 	at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
      2024-02-21T17:24:41.9688311Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:454)
      2024-02-21T17:24:41.9689673Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:204)
      2024-02-21T17:24:41.9691011Z 	at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27)
      2024-02-21T17:24:41.9692502Z 	at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:423)
      2024-02-21T17:24:41.9693842Z 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      2024-02-21T17:24:41.9695156Z 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      2024-02-21T17:24:41.9696266Z 	at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-21T17:24:41.9697616Z Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-21T17:24:41.9699204Z 	at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
      2024-02-21T17:24:41.9700707Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-21T17:24:41.9702162Z 	at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-21T17:24:41.9703358Z 	at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-21T17:24:41.9704786Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-21T17:24:41.9706528Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-21T17:24:41.9708208Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-21T17:24:41.9709771Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-21T17:24:41.9711172Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-21T17:24:41.9712388Z 	... 1 common frames omitted
      2024-02-21T17:24:41.9713163Z Caused by: io.debezium.DebeziumException: Error processing binlog event
      2024-02-21T17:24:41.9714012Z 	... 9 common frames omitted
      2024-02-21T17:24:41.9715308Z Caused by: io.debezium.DebeziumException: Database error while executing incremental snapshot for table 'null'
      2024-02-21T17:24:41.9717784Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:418)
      2024-02-21T17:24:41.9721102Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:130)
      2024-02-21T17:24:41.9724562Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-21T17:24:41.9727818Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-21T17:24:41.9730309Z 	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-21T17:24:41.9732338Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-21T17:24:41.9734612Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-21T17:24:41.9736712Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-21T17:24:41.9738209Z 	... 8 common frames omitted
      2024-02-21T17:24:41.9740746Z Caused by: java.lang.NullPointerException: Cannot invoke "io.debezium.pipeline.source.snapshot.incremental.DataCollection.getId()" because the return value of "io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext.currentDataCollectionId()" is null
      2024-02-21T17:24:41.9744684Z 	at io.debezium.pipeline.notification.IncrementalSnapshotNotificationService.notifyInProgress(IncrementalSnapshotNotificationService.java:132)
      2024-02-21T17:24:41.9747640Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.readChunk(AbstractIncrementalSnapshotChangeEventSource.java:405)
      2024-02-21T17:24:41.9749768Z 	... 15 common frames omitted
      2024-02-21T17:24:41.9751738Z 2024-02-21 17:24:41,855 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'POLLING_TASKS' to 'STOPPING'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-21T17:24:41.9755010Z 2024-02-21 17:24:41,950 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:24:41.9757652Z 2024-02-21 17:24:41,950 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the engine   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:24:41.9760262Z 2024-02-21 17:24:41,953 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:24:41.9762837Z 2024-02-21 17:24:41,953 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-21T17:24:41.9766307Z 2024-02-21 17:24:41,954 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Test io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal failed   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              vjuranek@redhat.com Vojtech Juranek
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: