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

ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal fails randomly

XMLWordPrintable

    • False
    • None
    • False

      ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal fails randomly (e.g. here|https://github.com/debezium/debezium/actions/runs/8066575804/job/22035468302?pr=5328) with

      2024-02-27T17:10:25.5580103Z 2024-02-27 17:10:25,439 ERROR  ReadOnlyIncrementalSnapshotIT|||engine|  Engine has failed with    [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:10:25.5595846Z 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-27T17:10:25.5597949Z 	at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
      2024-02-27T17:10:25.5599112Z 	at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
      2024-02-27T17:10:25.5600486Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:460)
      2024-02-27T17:10:25.5602330Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:204)
      2024-02-27T17:10:25.5603846Z 	at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27)
      2024-02-27T17:10:25.5605466Z 	at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:423)
      2024-02-27T17:10:25.5607190Z 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      2024-02-27T17:10:25.5608726Z 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      2024-02-27T17:10:25.5609858Z 	at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-27T17:10:25.5611922Z Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-27T17:10:25.5613677Z 	at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
      2024-02-27T17:10:25.5615456Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-27T17:10:25.5617079Z 	at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-27T17:10:25.5618235Z 	at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-27T17:10:25.5619789Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-27T17:10:25.5622018Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-27T17:10:25.5623917Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-27T17:10:25.5625424Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-27T17:10:25.5626743Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-27T17:10:25.5627656Z 	... 1 common frames omitted
      2024-02-27T17:10:25.5628398Z Caused by: io.debezium.DebeziumException: Error processing binlog event
      2024-02-27T17:10:25.5629178Z 	... 9 common frames omitted
      2024-02-27T17:10:25.5632073Z 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-27T17:10:25.5635972Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendEvent(AbstractIncrementalSnapshotChangeEventSource.java:182)
      2024-02-27T17:10:25.5639084Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:221)
      2024-02-27T17:10:25.5642535Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5645699Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendWindowEvents(AbstractIncrementalSnapshotChangeEventSource.java:174)
      2024-02-27T17:10:25.5649903Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:129)
      2024-02-27T17:10:25.5653755Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-27T17:10:25.5657072Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5659525Z 	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-27T17:10:25.5661785Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-27T17:10:25.5663943Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-27T17:10:25.5665977Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-27T17:10:25.5668053Z 	... 8 common frames omitted
      

      Whole test log:

      2024-02-27T17:09:35.1717680Z 2024-02-27 17:09:35,104 INFO   NotificationsIT|||test|  Starting test io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-27T17:09:35.1721865Z 2024-02-27 17:09:35,104 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-27T17:09:35.2131508Z 17:09:35.212 mysql-gtids2024-02-27T17:09:35.212210Z 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-27T17:09:35.2698510Z 2024-02-27 17:09:35,183 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-27T17:09:35.6517270Z 17:09:35.650 mysql-gtids2024-02-27T17:09:35.650665Z 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-27T17:09:35.6623401Z 17:09:35.660 mysql-gtids2024-02-27T17:09:35.660311Z 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-27T17:09:35.6710984Z 2024-02-27 17:09:35,642 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-27T17:09:35.6713868Z 2024-02-27 17:09:35,645 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'CREATING' to 'INITIALIZING'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.6724639Z 2024-02-27 17:09:35,646 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Loading the custom source info struct maker plugin: io.debezium.connector.mysql.MySqlSourceInfoStructMaker   [io.debezium.config.CommonConnectorConfig]
      2024-02-27T17:09:35.6728576Z 2024-02-27 17:09:35,647 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Using io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter   [io.debezium.connector.mysql.MySqlConnectorConfig]
      2024-02-27T17:09:35.6733496Z 2024-02-27 17:09:35,653 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-27T17:09:35.6745051Z 2024-02-27 17:09:35,654 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-27T17:09:35.6747366Z 2024-02-27 17:09:35,655 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'INITIALIZING' to 'CREATING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.6749956Z 2024-02-27 17:09:35,656 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'CREATING_TASKS' to 'STARTING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.6752260Z 2024-02-27 17:09:35,657 INFO   ||||  Starting MySqlConnectorTask with configuration:   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6754374Z 2024-02-27 17:09:35,657 INFO   ||||     connector.class = io.debezium.connector.mysql.MySqlConnector   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6757292Z 2024-02-27 17:09:35,657 INFO   ||||     message.key.columns = incremental_snapshot-test_7sc2b3.a42:pk1,pk2,pk3,pk4   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6759252Z 2024-02-27 17:09:35,657 INFO   ||||     incremental.snapshot.chunk.size = 1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6761118Z 2024-02-27 17:09:35,657 INFO   ||||     record.processing.shutdown.timeout.ms = 1000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6763020Z 2024-02-27 17:09:35,657 INFO   ||||     signal.enabled.channels = source,kafka   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6764764Z 2024-02-27 17:09:35,657 INFO   ||||     include.schema.changes = false   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6766504Z 2024-02-27 17:09:35,657 INFO   ||||     record.processing.order = ORDERED   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6768774Z 2024-02-27 17:09:35,657 INFO   ||||     connector.adapter = mysql   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6771617Z 2024-02-27 17:09:35,657 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-27T17:09:35.6774900Z 2024-02-27 17:09:35,657 INFO   ||||     topic.prefix = is_test   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6776985Z 2024-02-27 17:09:35,657 INFO   ||||     database.jdbc.driver = com.mysql.cj.jdbc.Driver   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6779850Z 2024-02-27 17:09:35,657 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-27T17:09:35.6782398Z 2024-02-27 17:09:35,657 INFO   ||||     poll.interval.ms = 10   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6784458Z 2024-02-27 17:09:35,657 INFO   ||||     signal.data.collection = incremental_snapshot-test_7sc2b3.debezium_signal   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6786996Z 2024-02-27 17:09:35,657 INFO   ||||     record.processing.threads = 4   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6789461Z 2024-02-27 17:09:35,657 INFO   ||||     errors.retry.delay.initial.ms = 300   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6791484Z 2024-02-27 17:09:35,657 INFO   ||||     binlog.buffer.size = 10000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6793374Z 2024-02-27 17:09:35,657 INFO   ||||     value.converter = org.apache.kafka.connect.json.JsonConverter   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6795612Z 2024-02-27 17:09:35,657 INFO   ||||     key.converter = org.apache.kafka.connect.json.JsonConverter   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6797782Z 2024-02-27 17:09:35,657 INFO   ||||     database.user = mysqluser   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6799892Z 2024-02-27 17:09:35,657 INFO   ||||     signal.kafka.bootstrap.servers = localhost:34677   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6801848Z 2024-02-27 17:09:35,657 INFO   ||||     database.server.id = 18765   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6804083Z 2024-02-27 17:09:35,657 INFO   ||||     offset.storage = org.apache.kafka.connect.storage.FileOffsetBackingStore   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6806569Z 2024-02-27 17:09:35,657 INFO   ||||     signal.kafka.topic = incremental_snapshot-test_7sc2b3signals_topic   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6809004Z 2024-02-27 17:09:35,657 INFO   ||||     signal.poll.interval.ms = 1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6810711Z 2024-02-27 17:09:35,657 INFO   ||||     read.only = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6812656Z 2024-02-27 17:09:35,657 INFO   ||||     offset.flush.timeout.ms = 5000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6814699Z 2024-02-27 17:09:35,657 INFO   ||||     errors.retry.delay.max.ms = 10000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6816897Z 2024-02-27 17:09:35,657 INFO   ||||     incremental.snapshot.allow.schema.changes = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6818838Z 2024-02-27 17:09:35,657 INFO   ||||     database.port = 4302   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6820582Z 2024-02-27 17:09:35,657 INFO   ||||     offset.flush.interval.ms = 0   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6822374Z 2024-02-27 17:09:35,657 INFO   ||||     database.ssl.mode = disabled   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6824863Z 2024-02-27 17:09:35,657 INFO   ||||     internal.task.management.timeout.ms = 180000   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6827295Z 2024-02-27 17:09:35,657 INFO   ||||     database.protocol = jdbc:mysql   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6831670Z 2024-02-27 17:09:35,657 INFO   ||||     schema.history.internal = io.debezium.storage.file.history.FileSchemaHistory   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6834872Z 2024-02-27 17:09:35,657 INFO   ||||     record.processing.with.serial.consumer = false   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6840703Z 2024-02-27 17:09:35,657 INFO   ||||     table.exclude.list = incremental_snapshot-test_7sc2b3.b   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6842734Z 2024-02-27 17:09:35,657 INFO   ||||     errors.max.retries = -1   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6844512Z 2024-02-27 17:09:35,657 INFO   ||||     database.hostname = localhost   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6846302Z 2024-02-27 17:09:35,657 INFO   ||||     database.password = ********   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6848335Z 2024-02-27 17:09:35,657 INFO   ||||     schema.name.adjustment.mode = avro   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6850164Z 2024-02-27 17:09:35,657 INFO   ||||     name = testing-connector   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6851882Z 2024-02-27 17:09:35,657 INFO   ||||     include.query = true   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6853473Z 2024-02-27 17:09:35,657 INFO   ||||     snapshot.mode = schema_only   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6855375Z 2024-02-27 17:09:35,657 INFO   ||||     database.include.list = incremental_snapshot-test_7sc2b3   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6857526Z 2024-02-27 17:09:35,657 INFO   ||||  Loading the custom source info struct maker plugin: io.debezium.connector.mysql.MySqlSourceInfoStructMaker   [io.debezium.config.CommonConnectorConfig]
      2024-02-27T17:09:35.6859762Z 2024-02-27 17:09:35,657 INFO   ||||  Using io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter   [io.debezium.connector.mysql.MySqlConnectorConfig]
      2024-02-27T17:09:35.6862435Z 2024-02-27 17:09:35,658 INFO   ||||  Loading the custom topic naming strategy plugin: io.debezium.schema.DefaultTopicNamingStrategy   [io.debezium.config.CommonConnectorConfig]
      2024-02-27T17:09:35.6864537Z 2024-02-27 17:09:35,659 INFO   ||||  No previous offsets found   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:09:35.6867150Z 17:09:35.683 mysql-gtids2024-02-27T17:09:35.683507Z 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-27T17:09:35.7710936Z 2024-02-27 17:09:35,678 INFO   ||||  Closing connection before starting schema recovery   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-27T17:09:35.7714039Z 2024-02-27 17:09:35,679 INFO   ||||  Connection gracefully closed   [io.debezium.jdbc.JdbcConnection]
      2024-02-27T17:09:35.7717804Z 2024-02-27 17:09:35,679 INFO   ||||  Connector started for the first time, database schema history recovery will not be executed   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-27T17:09:35.7721107Z 2024-02-27 17:09:35,679 INFO   ||||  Reconnecting after finishing schema recovery   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-27T17:09:35.7732595Z 2024-02-27 17:09:35,685 INFO   ||||  No previous offset found   [io.debezium.connector.mysql.MySqlConnectorTask]
      2024-02-27T17:09:35.7735281Z 2024-02-27 17:09:35,685 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = SignalProcessor   [io.debezium.util.Threads]
      2024-02-27T17:09:35.7739251Z 2024-02-27 17:09:35,719 INFO   ||||  Subscribing to signals topic 'incremental_snapshot-test_7sc2b3signals_topic'   [io.debezium.pipeline.signal.channels.KafkaSignalChannel]
      2024-02-27T17:09:35.7742655Z 2024-02-27 17:09:35,720 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = change-event-source-coordinator   [io.debezium.util.Threads]
      2024-02-27T17:09:35.7746094Z 2024-02-27 17:09:35,720 INFO   ||||  Requested thread factory for connector MySqlConnector, id = is_test named = blocking-snapshot   [io.debezium.util.Threads]
      2024-02-27T17:09:35.7749407Z 2024-02-27 17:09:35,721 INFO   ||||  Creating thread debezium-mysqlconnector-is_test-change-event-source-coordinator   [io.debezium.util.Threads]
      2024-02-27T17:09:35.7753164Z 2024-02-27 17:09:35,721 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  All tasks have stated successfully.   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.7756690Z 2024-02-27 17:09:35,721 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'STARTING_TASKS' to 'POLLING_TASKS'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.7760984Z 2024-02-27 17:09:35,721 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Using io.debezium.embedded.async.ParallelSmtConsumerProcessor processor   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:09:35.9712543Z 2024-02-27 17:09:35,873 INFO   MySQL|is_test|0|snapshot|  Read binlog position of MySQL primary server   [io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter]
      2024-02-27T17:09:35.9716658Z 2024-02-27 17:09:35,874 INFO   MySQL|is_test|0|snapshot|  	 using binlog 'mysql-bin.000002' at position '12529944' and gtid '7da85286-d590-11ee-899d-0242ac110002:1-16028'   [io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter]
      2024-02-27T17:09:37.3714144Z 17:09:37.370 mysql-gtids2024-02-27T17:09:37.370508Z 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-27T17:09:37.3748327Z 2024-02-27 17:09:37,356 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3752868Z 2024-02-27 17:09:37,356 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3757317Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3762130Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a4.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a4.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3766684Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a4.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a4.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3771304Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a4.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a4.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3775768Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a42.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a42.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3780441Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a42.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a42.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3784917Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a42.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a42.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3789427Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_date.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_date.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3793878Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_date.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_date.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3798423Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_date.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_date.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3802919Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_dt.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_dt.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3807501Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_dt.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_dt.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3811997Z 2024-02-27 17:09:37,357 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.a_dt.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.a_dt.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3816455Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.c.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.c.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.3820952Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.c.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.c.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.4007878Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.c.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.c.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.4012595Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.debezium_signal.Value' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.debezium_signal.Value'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.4017518Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.debezium_signal.Key' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.debezium_signal.Key'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.4022261Z 2024-02-27 17:09:37,358 WARN   MySQL|is_test|0|snapshot|  The Kafka Connect schema name 'is_test.incremental_snapshot-test_7sc2b3.debezium_signal.Envelope' is not a valid Avro schema name, so replacing with 'is_test.incremental_snapshot_test_7sc2b3.debezium_signal.Envelope'   [io.debezium.schema.SchemaNameAdjuster]
      2024-02-27T17:09:37.4025728Z 2024-02-27 17:09:37,359 INFO   MySQL|is_test|0|snapshot|  Snapshot - Final stage   [io.debezium.pipeline.source.AbstractSnapshotChangeEventSource]
      2024-02-27T17:09:37.4027993Z 2024-02-27 17:09:37,359 INFO   MySQL|is_test|0|snapshot|  Snapshot completed   [io.debezium.pipeline.source.AbstractSnapshotChangeEventSource]
      2024-02-27T17:09:37.4030450Z 2024-02-27 17:09:37,359 INFO   MySQL|is_test|0|streaming|  Requested thread factory for connector MySqlConnector, id = is_test named = binlog-client   [io.debezium.util.Threads]
      2024-02-27T17:09:37.4033641Z 2024-02-27 17:09:37,367 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-27T17:09:37.4036636Z 2024-02-27 17:09:37,368 INFO   MySQL|is_test|0|streaming|  SignalProcessor started. Scheduling it every 1ms   [io.debezium.pipeline.signal.SignalProcessor]
      2024-02-27T17:09:37.4038977Z 2024-02-27 17:09:37,368 INFO   MySQL|is_test|0|streaming|  Creating thread debezium-mysqlconnector-is_test-SignalProcessor   [io.debezium.util.Threads]
      2024-02-27T17:09:37.4041296Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  GTID set purged on server: ''   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:37.4043743Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Attempting to generate a filtered GTID set   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4046651Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  GTID set from previous recorded offset: 7da85286-d590-11ee-899d-0242ac110002:1-16028   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4062987Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  GTID set available on server: 7da85286-d590-11ee-899d-0242ac110002:1-16028   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4066081Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Using first available positions for new GTID channels   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4069207Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Relevant GTID set available on server: 7da85286-d590-11ee-899d-0242ac110002:1-16028   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4072811Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Final merged GTID set to use when connecting to MySQL: 7da85286-d590-11ee-899d-0242ac110002:1-16028   [io.debezium.connector.mysql.strategy.mysql.MySqlConnection]
      2024-02-27T17:09:37.4076334Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Registering binlog reader with GTID set: '7da85286-d590-11ee-899d-0242ac110002:1-16028'   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:37.4079211Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Skip 0 events on streaming start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:37.4081690Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Skip 0 rows on streaming start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:37.4084080Z 2024-02-27 17:09:37,369 INFO   MySQL|is_test|0|streaming|  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-27T17:09:37.4086431Z 2024-02-27 17:09:37,370 INFO   ||||  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-27T17:09:37.4727500Z Feb 27, 2024 5:09:37 PM com.github.shyiko.mysql.binlog.BinaryLogClient connect
      2024-02-27T17:09:37.4729245Z INFO: Connected to localhost:4302 at 7da85286-d590-11ee-899d-0242ac110002:1-16028 (sid:18765, cid:1707)
      2024-02-27T17:09:37.4739931Z 2024-02-27 17:09:37,375 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=12529944, currentRowNumber=0, serverId=0, sourceTime=2024-02-27T17:09:37.358Z, threadId=-1, currentQuery=null, tableIds=[incremental_snapshot-test_7sc2b3.debezium_signal], databaseName=year_test_77e881], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=7da85286-d590-11ee-899d-0242ac110002:1-16028, currentGtidSet=7da85286-d590-11ee-899d-0242ac110002:1-16028, restartBinlogFilename=mysql-bin.000002, restartBinlogPosition=12529944, 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-27T17:09:37.4750495Z 2024-02-27 17:09:37,375 INFO   MySQL|is_test|0|streaming|  Waiting for keepalive thread to start   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:37.4753030Z 2024-02-27 17:09:37,375 INFO   MySQL|is_test||binlog|  Creating thread debezium-mysqlconnector-is_test-binlog-client   [io.debezium.util.Threads]
      2024-02-27T17:09:37.5728284Z 2024-02-27 17:09:37,475 INFO   MySQL|is_test|0|streaming|  Keepalive thread is running   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:09:41.9761705Z 2024-02-27 17:09:41,888 INFO   ||||  Requested 'INCREMENTAL' snapshot of data collections '[incremental_snapshot-test_7sc2b3.a]' with additional conditions '[]' and surrogate key 'PK of table will be used'   [io.debezium.pipeline.signal.actions.snapshotting.ExecuteSnapshot]
      2024-02-27T17:10:25.4088795Z 2024-02-27 17:10:25,407 INFO   MySQL|is_test||binlog|  Incremental snapshot's schema verification passed = true, schema = columns: {
      2024-02-27T17:10:25.4090060Z   pk INT(10, 0) NOT NULL
      2024-02-27T17:10:25.4092669Z   aa INT(10, 0) DEFAULT VALUE NULL
      2024-02-27T17:10:25.4093428Z }
      2024-02-27T17:10:25.4095640Z primary key: []
      2024-02-27T17:10:25.4096315Z default charset: null
      2024-02-27T17:10:25.4096797Z comment: null
      2024-02-27T17:10:25.4097192Z attributes: {
      2024-02-27T17:10:25.4097555Z }
      2024-02-27T17:10:25.4098474Z    [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext]
      2024-02-27T17:10:25.4101803Z 2024-02-27 17:10:25,407 INFO   MySQL|is_test||binlog|  Incremental snapshot for table 'incremental_snapshot-test_7sc2b3.a' will end at position [1000]   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-27T17:10:25.5489761Z 2024-02-27 17:10:25,417 INFO   ||||  5 records sent during previous 00:00:49.761, last recorded offset of {server=is_test} partition is {incremental_snapshot_correlation_id=null, ts_sec=1709053825, file=mysql-bin.000002, pos=12529944, incremental_snapshot_signal_offset=0, incremental_snapshot_maximum_key=aced0005757200135b4c6a6176612e6c616e672e4f626a6563743b90ce589f1073296c020000787000000001737200116a6176612e6c616e672e496e746567657212e2a0a4f781873802000149000576616c7565787200106a6176612e6c616e672e4e756d62657286ac951d0b94e08b0200007870000003e8, gtids=7da85286-d590-11ee-899d-0242ac110002:1-16028, server_id=112233, incremental_snapshot_collections=[{"incremental_snapshot_collections_id":"incremental_snapshot-test_7sc2b3.a","incremental_snapshot_collections_additional_condition":null,"incremental_snapshot_collections_surrogate_key":null}], incremental_snapshot_primary_key=aced0005757200135b4c6a6176612e6c616e672e4f626a6563743b90ce589f1073296c020000787000000001737200116a6176612e6c616e672e496e746567657212e2a0a4f781873802000149000576616c7565787200106a6176612e6c616e672e4e756d62657286ac951d0b94e08b020000787000000005}   [io.debezium.connector.common.BaseSourceTask]
      2024-02-27T17:10:25.5500679Z 2024-02-27 17:10:25,435 INFO   ||||  Requested stop of snapshot 'INCREMENTAL' for data collections '[incremental_snapshot-test_7sc2b3.a]'   [io.debezium.pipeline.signal.actions.snapshotting.StopSnapshot]
      2024-02-27T17:10:25.5504199Z 2024-02-27 17:10:25,435 INFO   ||||  Removing '[incremental_snapshot-test_7sc2b3.a]' collections from incremental snapshot   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-27T17:10:25.5507990Z 2024-02-27 17:10:25,435 INFO   ||||  Removed 'incremental_snapshot-test_7sc2b3.a' from incremental snapshot collection list.   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-27T17:10:25.5511605Z 2024-02-27 17:10:25,435 INFO   ||||  Advancing to next available collection in the incremental snapshot.   [io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource]
      2024-02-27T17:10:25.5516426Z 2024-02-27 17:10:25,436 ERROR  MySQL|is_test||binlog|  Error during binlog processing. Last offset stored = {ts_sec=1709053777, file=mysql-bin.000002, pos=12529944, incremental_snapshot_signal_offset=null, gtids=7da85286-d590-11ee-899d-0242ac110002:1-16028, server_id=112233, event=1}, binlog reader near position = mysql-bin.000002/12529944   [io.debezium.connector.mysql.MySqlStreamingChangeEventSource]
      2024-02-27T17:10:25.5520121Z 2024-02-27 17:10:25,436 ERROR  MySQL|is_test||binlog|  Producer failure   [io.debezium.pipeline.ErrorHandler]
      2024-02-27T17:10:25.5521389Z io.debezium.DebeziumException: Error processing binlog event
      2024-02-27T17:10:25.5522918Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-27T17:10:25.5524597Z 	at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-27T17:10:25.5525810Z 	at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-27T17:10:25.5527689Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-27T17:10:25.5529591Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-27T17:10:25.5531299Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-27T17:10:25.5532837Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-27T17:10:25.5534198Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-27T17:10:25.5535210Z 	at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-27T17:10:25.5538124Z 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-27T17:10:25.5542247Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendEvent(AbstractIncrementalSnapshotChangeEventSource.java:182)
      2024-02-27T17:10:25.5545378Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:221)
      2024-02-27T17:10:25.5549850Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5553484Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendWindowEvents(AbstractIncrementalSnapshotChangeEventSource.java:174)
      2024-02-27T17:10:25.5556858Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:129)
      2024-02-27T17:10:25.5560245Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-27T17:10:25.5563486Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5565936Z 	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-27T17:10:25.5568104Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-27T17:10:25.5570300Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-27T17:10:25.5572467Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-27T17:10:25.5573856Z 	... 8 common frames omitted
      2024-02-27T17:10:25.5576710Z 2024-02-27 17:10:25,436 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-27T17:10:25.5580103Z 2024-02-27 17:10:25,439 ERROR  ReadOnlyIncrementalSnapshotIT|||engine|  Engine has failed with    [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:10:25.5595846Z 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-27T17:10:25.5597949Z 	at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
      2024-02-27T17:10:25.5599112Z 	at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
      2024-02-27T17:10:25.5600486Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.runTasksPolling(AsyncEmbeddedEngine.java:460)
      2024-02-27T17:10:25.5602330Z 	at io.debezium.embedded.async.AsyncEmbeddedEngine.run(AsyncEmbeddedEngine.java:204)
      2024-02-27T17:10:25.5603846Z 	at io.debezium.embedded.async.TestingAsyncEmbeddedEngine.run(TestingAsyncEmbeddedEngine.java:27)
      2024-02-27T17:10:25.5605466Z 	at io.debezium.embedded.AbstractConnectorTest.lambda$start$8(AbstractConnectorTest.java:423)
      2024-02-27T17:10:25.5607190Z 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      2024-02-27T17:10:25.5608726Z 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      2024-02-27T17:10:25.5609858Z 	at java.base/java.lang.Thread.run(Thread.java:840)
      2024-02-27T17:10:25.5611922Z Caused by: org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2024-02-27T17:10:25.5613677Z 	at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:67)
      2024-02-27T17:10:25.5615456Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)
      2024-02-27T17:10:25.5617079Z 	at io.debezium.connector.mysql.EventBuffer.consumeEvent(EventBuffer.java:178)
      2024-02-27T17:10:25.5618235Z 	at io.debezium.connector.mysql.EventBuffer.add(EventBuffer.java:124)
      2024-02-27T17:10:25.5619789Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$31(MySqlStreamingChangeEventSource.java:888)
      2024-02-27T17:10:25.5622018Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1263)
      2024-02-27T17:10:25.5623917Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1089)
      2024-02-27T17:10:25.5625424Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:648)
      2024-02-27T17:10:25.5626743Z 	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:949)
      2024-02-27T17:10:25.5627656Z 	... 1 common frames omitted
      2024-02-27T17:10:25.5628398Z Caused by: io.debezium.DebeziumException: Error processing binlog event
      2024-02-27T17:10:25.5629178Z 	... 9 common frames omitted
      2024-02-27T17:10:25.5632073Z 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-27T17:10:25.5635972Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendEvent(AbstractIncrementalSnapshotChangeEventSource.java:182)
      2024-02-27T17:10:25.5639084Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:221)
      2024-02-27T17:10:25.5642535Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.sendEvent(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5645699Z 	at io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource.sendWindowEvents(AbstractIncrementalSnapshotChangeEventSource.java:174)
      2024-02-27T17:10:25.5649903Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.readUntilGtidChange(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:129)
      2024-02-27T17:10:25.5653755Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:122)
      2024-02-27T17:10:25.5657072Z 	at io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotChangeEventSource.processHeartbeat(MySqlReadOnlyIncrementalSnapshotChangeEventSource.java:83)
      2024-02-27T17:10:25.5659525Z 	at io.debezium.pipeline.EventDispatcher.dispatchServerHeartbeatEvent(EventDispatcher.java:441)
      2024-02-27T17:10:25.5661785Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleServerHeartbeat(MySqlStreamingChangeEventSource.java:347)
      2024-02-27T17:10:25.5663943Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$14(MySqlStreamingChangeEventSource.java:850)
      2024-02-27T17:10:25.5665977Z 	at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)
      2024-02-27T17:10:25.5668053Z 	... 8 common frames omitted
      2024-02-27T17:10:25.5669901Z 2024-02-27 17:10:25,439 INFO   ReadOnlyIncrementalSnapshotIT|||engine|  Engine state has changed from 'POLLING_TASKS' to 'STOPPING'   [io.debezium.embedded.async.AsyncEmbeddedEngine]
      2024-02-27T17:10:25.5672955Z 2024-02-27 17:10:25,537 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-27T17:10:25.5675571Z 2024-02-27 17:10:25,538 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the engine   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-27T17:10:25.5678127Z 2024-02-27 17:10:25,542 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Stopping the connector   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      2024-02-27T17:10:25.5682452Z 2024-02-27 17:10:25,543 INFO   ReadOnlyIncrementalSnapshotIT|||test|  Test io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT#testStopSnapshotKafkaSignal failed   [io.debezium.connector.mysql.ReadOnlyIncrementalSnapshotIT]
      

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

              Created:
              Updated: