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

MySQL Debezium Connector 2.2.0.Final failure - No TableMapEventData has been found for table id

    XMLWordPrintable

Details

    • Bug
    • Resolution: Obsolete
    • Major
    • None
    • 2.2.0.Final
    • mysql-connector
    • None
    • Critical

    Description

      Bug report

      It seems that it's a bug from the MySQL binlog library used by Debezium MySQL Connector. The lib can not find the table in the Event for some reason. So far, we can't find any workaround to it.

      What Debezium connector do you use and what version?

      MySQL Debezium Connector 2.2.0.Final

      What is the connector configuration?

      All components (Broker, Connect Worker, Schema Registry, MySQL etc are deployed locally on docker-compose).

       

      {
        "name": "source-mysql-local2",
        "config":
        {
          "connector.class": "io.debezium.connector.mysql.MySqlConnector",
          "tasks.max": "1",
          "database.hostname": "taxify-binlog-db",
          "database.port": "3306",
          "database.user": "root",
          "database.password": "",
          "database.server.id": "184055",
          "include.schema.changes": "true",
          "snapshot.mode": "schema_only",
          "decimal.handling.mode": "string",
          "max.queue.size": "24576",
          "max.batch.size": "6144",
          "topic.prefix": "cdc_mysql_avro.local2",
          "schema.history.internal.kafka.topic": "my_schema_history2",
          "schema.history.internal.kafka.bootstrap.servers": "kafka:29092"
        }
      } 

       

       

      What is the captured database version and mode of deployment?

      MySQL version:

       

      mysql> select version();
      +-----------+
      | version() |
      +-----------+
      | 8.0.22-13 |
      +-----------+ 

       

      MySQL is deployed in local docker-compose.

       

      What behaviour do you expect?

      In MySQL, create the table, insert a row (call in row-a):

      CREATE DATABASE db1;
      USE db1;
      
      CREATE TABLE `t1` (`id` bigint(20) NOT NULL AUTO_INCREMENT,`created` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,`last_update` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,`c4` enum('e1') NOT NULL DEFAULT 'e1',`c5` varchar(255) NOT NULL,`c6` int(11) DEFAULT NULL,`c7` enum('e1','e2') NOT NULL DEFAULT 'e1',`c8` enum('e1','e2','e3','e4','e5','e6','e7','e8','e9') NOT NULL DEFAULT 'e1',`c9` enum('e1','e2','e3','e4','e5','e6') NOT NULL DEFAULT 'e1',`c10` varchar(255) DEFAULT NULL,`c11` char(32) DEFAULT NULL,`c12` char(128) DEFAULT NULL,`c13` char(8) DEFAULT NULL,`c14` varchar(255) DEFAULT NULL,`c15` varchar(255) DEFAULT NULL,`c16` varchar(50) DEFAULT NULL,`c17` date DEFAULT NULL,`c18` varchar(255) DEFAULT NULL,`c19` enum('e1','e2') DEFAULT NULL,`c20` timestamp NULL DEFAULT NULL,`c21` varchar(20) DEFAULT NULL,`c22` int(11) DEFAULT NULL,`c23` char(2) DEFAULT NULL,`c24` int(11) DEFAULT NULL,`c25` varchar(255) DEFAULT NULL,`c26` varchar(2) DEFAULT NULL,`c27` varchar(45) DEFAULT NULL,`c28` tinyint(1) DEFAULT NULL,`c29` varchar(255) DEFAULT NULL,`c30` year(4) DEFAULT NULL,`c31` tinyint(4) DEFAULT NULL,`c32` varchar(50) DEFAULT NULL,`c33` varchar(50) DEFAULT NULL,`c34` varchar(30) DEFAULT NULL,`c35` varchar(20) DEFAULT NULL,`c36` tinyint(1) DEFAULT NULL,PRIMARY KEY (`id`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
      
      // insert row-a
      INSERT INTO `t1` (    `id`,    `created`,    `last_update`,    `c4`,    `c5`,    `c6`,    `c7`,    `c8`,    `c9`,    `c10`,    `c11`,    `c12`,    `c13`,    `c14`,    `c15`,    `c16`,    `c17`,    `c18`,    `c19`,    `c20`,    `c21`,    `c22`,    `c23`,    `c24`,    `c25`,    `c26`,    `c27`,    `c28`,    `c29`,    `c30`,    `c31`,    `c32`,    `c33`,    `c34`,    `c35`,    `c36`)VALUES (    2,    '2020-02-12 00:00:00',    '2020-02-12 00:00:00',    'e1',    'a',    1,    'e1',    'e1',    'e1',    'a',    'a',    'a',    'a',    'a',    'a',    'a',    '2020-02-12',    'a',    'e1',    '2020-02-12 00:00:00',    'a',    1,    'a',    1,    'a',    'a',    'a',    1,    'a',    2020,    1,    'a',    'a',    'a',    'a',    1    );

       

      After applying the following DDL to add a new `tinyint(1)` column and inserting a new row (call it row-b), the new row should be sent to Kafka.

      ALTER TABLE `db1`.`t1` ADD COLUMN `c37` tinyint(1) DEFAULT NULL;
      
      // insert row-b
      INSERT INTO `t1` (    `id`,    `created`,    `last_update`,    `c4`,    `c5`,    `c6`,    `c7`,    `c8`,    `c9`,    `c10`,    `c11`,    `c12`,    `c13`,    `c14`,    `c15`,    `c16`,    `c17`,    `c18`,    `c19`,    `c20`,    `c21`,    `c22`,    `c23`,    `c24`,    `c25`,    `c26`,    `c27`,    `c28`,    `c29`,    `c30`,    `c31`,    `c32`,    `c33`,    `c34`,    `c35`,    `c36`,    `c37`)VALUES (    3,    '2020-02-12 00:00:00',    '2020-02-12 00:00:00',    'e1',    'a',    1,    'e1',    'e1',    'e1',    'a',    'a',    'a',    'a',    'a',    'a',    'a',    '2020-02-12',    'a',    'e1',    '2020-02-12 00:00:00',    'a',    1,    'a',    1,    'a',    'a',    'a',    1,    'a',    2020,    1,    'a',    'a',    'a',    'a',    1,    1    ); 

      What behaviour do you see?

      After applying the DDL to add a new `tinyint(1)` column, the connector is still running. However, when inserting a new row called it row-b (after the DDL has been applied), the connector failed with the following error and stack trace:

       

      [2023-05-02 19:53:04,816] ERROR Error during binlog processing. Last offset stored = {transaction_id=null, ts_sec=1683057142, file=mysql-bin.000003, pos=2703, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-9, server_id=223344, event=1}, binlog reader near position = mysql-bin.000003/2785 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:53:04,843] ERROR Producer failure (io.debezium.pipeline.ErrorHandler)
      io.debezium.DebeziumException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1236)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onEventDeserializationFailure(MySqlStreamingChangeEventSource.java:1289)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1064)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:631)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:932)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:341)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeTableMapEventData(EventDeserializer.java:313)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.nextEvent(EventDeserializer.java:237)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$1.nextEvent(MySqlStreamingChangeEventSource.java:232)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1051)
          ... 3 more
      Caused by: java.io.IOException: Unsupported table metadata field type 0
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventMetadataDeserializer.deserialize(TableMapEventMetadataDeserializer.java:52)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:47)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:27)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:335)
          ... 7 more
      [2023-05-02 19:53:04,860] WARN Retry 1 of unlimited retries will be attempted (io.debezium.pipeline.ErrorHandler)
      [2023-05-02 19:53:04,861] ERROR Error during binlog processing. Last offset stored = {transaction_id=null, ts_sec=1683057142, file=mysql-bin.000003, pos=2703, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-9, server_id=223344, event=1}, binlog reader near position = mysql-bin.000003/2785 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:53:04,862] ERROR Producer failure (io.debezium.pipeline.ErrorHandler)
      io.debezium.DebeziumException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=EXT_WRITE_ROWS, serverId=223344, headerLength=19, dataLength=116, nextPosition=3064, flags=0}
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1236)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onEventDeserializationFailure(MySqlStreamingChangeEventSource.java:1289)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1064)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:631)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:932)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=EXT_WRITE_ROWS, serverId=223344, headerLength=19, dataLength=116, nextPosition=3064, flags=0}
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:341)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.nextEvent(EventDeserializer.java:244)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$1.nextEvent(MySqlStreamingChangeEventSource.java:232)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1051)
          ... 3 more
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.MissingTableMapEventException: No TableMapEventData has been found for table id:117. Usually that means that you have started reading binary log 'within the logical event group' (e.g. from WRITE_ROWS and not proceeding TABLE_MAP
          at com.github.shyiko.mysql.binlog.event.deserialization.AbstractRowsEventDataDeserializer.deserializeRow(AbstractRowsEventDataDeserializer.java:109)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserializeRows(WriteRowsEventDataDeserializer.java:64)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserialize(WriteRowsEventDataDeserializer.java:56)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserialize(WriteRowsEventDataDeserializer.java:32)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:335)
          ... 6 more 

       

       

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

      I've tested it against the latest stable Debezium version 2.2.0 Final.

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

       

      [2023-05-02 19:50:23,951] INFO Kafka version: 6.1.1-ccs (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:23,951] INFO Kafka commitId: c209f70c6c2e52ae (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:23,951] INFO Kafka startTimeMs: 1683057023951 (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,001] INFO [Producer clientId=connector-producer-source-mysql-local2-0] Cluster ID: LgjEAa6RQUmsbH92o0eZsw (org.apache.kafka.clients.Metadata)
      [2023-05-02 19:50:24,002] INFO [Worker clientId=connect-1, groupId=kafka-connect] Finished starting connectors and tasks (org.apache.kafka.connect.runtime.distributed.DistributedHerder)
      [2023-05-02 19:50:24,057] INFO Starting MySqlConnectorTask with configuration: (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,067] INFO    connector.class = io.debezium.connector.mysql.MySqlConnector (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,068] INFO    database.user = root (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,068] INFO    max.queue.size = 24576 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,071] INFO    database.server.id = 184055 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,071] INFO    tasks.max = 1 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,071] INFO    schema.history.internal.kafka.bootstrap.servers = kafka:29092 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    database.port = 3306 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    include.schema.changes = true (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    topic.prefix = cdc_mysql_avro.local2 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    decimal.handling.mode = string (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    schema.history.internal.kafka.topic = my_schema_history2 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,072] INFO    task.class = io.debezium.connector.mysql.MySqlConnectorTask (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,073] INFO    database.hostname = taxify-binlog-db (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,073] INFO    database.password = ******** (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,073] INFO    name = source-mysql-local2 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,073] INFO    max.batch.size = 6144 (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,073] INFO    snapshot.mode = schema_only (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,074] INFO WorkerSourceTask{id=source-mysql-local2-0} Source task finished initialization and start (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:50:24,074] INFO Attempting to start task (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,209] INFO Loading the custom topic naming strategy plugin: io.debezium.schema.DefaultTopicNamingStrategy (io.debezium.config.CommonConnectorConfig)
      [2023-05-02 19:50:24,436] INFO No previous offsets found (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:24,680] INFO KafkaSchemaHistory Consumer config: {key.deserializer=org.apache.kafka.common.serialization.StringDeserializer, value.deserializer=org.apache.kafka.common.serialization.StringDeserializer, enable.auto.commit=false, group.id=cdc_mysql_avro.local2-schemahistory, bootstrap.servers=kafka:29092, fetch.min.bytes=1, session.timeout.ms=10000, auto.offset.reset=earliest, client.id=cdc_mysql_avro.local2-schemahistory} (io.debezium.storage.kafka.history.KafkaSchemaHistory)
      [2023-05-02 19:50:24,681] INFO KafkaSchemaHistory Producer config: {retries=1, value.serializer=org.apache.kafka.common.serialization.StringSerializer, acks=1, batch.size=32768, max.block.ms=10000, bootstrap.servers=kafka:29092, buffer.memory=1048576, key.serializer=org.apache.kafka.common.serialization.StringSerializer, client.id=cdc_mysql_avro.local2-schemahistory, linger.ms=0} (io.debezium.storage.kafka.history.KafkaSchemaHistory)
      [2023-05-02 19:50:24,684] INFO Requested thread factory for connector MySqlConnector, id = cdc_mysql_avro.local2 named = db-history-config-check (io.debezium.util.Threads)
      [2023-05-02 19:50:24,699] INFO ProducerConfig values:
          acks = 1
          batch.size = 32768
          bootstrap.servers = [kafka:29092]
          buffer.memory = 1048576
          client.dns.lookup = use_all_dns_ips
          client.id = cdc_mysql_avro.local2-schemahistory
          compression.type = none
          connections.max.idle.ms = 540000
          delivery.timeout.ms = 120000
          enable.idempotence = false
          interceptor.classes = []
          internal.auto.downgrade.txn.commit = false
          key.serializer = class org.apache.kafka.common.serialization.StringSerializer
          linger.ms = 0
          max.block.ms = 10000
          max.in.flight.requests.per.connection = 5
          max.request.size = 1048576
          metadata.max.age.ms = 300000
          metadata.max.idle.ms = 300000
          metric.reporters = []
          metrics.num.samples = 2
          metrics.recording.level = INFO
          metrics.sample.window.ms = 30000
          partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
          receive.buffer.bytes = 32768
          reconnect.backoff.max.ms = 1000
          reconnect.backoff.ms = 50
          request.timeout.ms = 30000
          retries = 1
          retry.backoff.ms = 100
          sasl.client.callback.handler.class = null
          sasl.jaas.config = null
          sasl.kerberos.kinit.cmd = /usr/bin/kinit
          sasl.kerberos.min.time.before.relogin = 60000
          sasl.kerberos.service.name = null
          sasl.kerberos.ticket.renew.jitter = 0.05
          sasl.kerberos.ticket.renew.window.factor = 0.8
          sasl.login.callback.handler.class = null
          sasl.login.class = null
          sasl.login.refresh.buffer.seconds = 300
          sasl.login.refresh.min.period.seconds = 60
          sasl.login.refresh.window.factor = 0.8
          sasl.login.refresh.window.jitter = 0.05
          sasl.mechanism = GSSAPI
          security.protocol = PLAINTEXT
          security.providers = null
          send.buffer.bytes = 131072
          socket.connection.setup.timeout.max.ms = 127000
          socket.connection.setup.timeout.ms = 10000
          ssl.cipher.suites = null
          ssl.enabled.protocols = [TLSv1.2, TLSv1.3]
          ssl.endpoint.identification.algorithm = https
          ssl.engine.factory.class = null
          ssl.key.password = null
          ssl.keymanager.algorithm = SunX509
          ssl.keystore.certificate.chain = null
          ssl.keystore.key = null
          ssl.keystore.location = null
          ssl.keystore.password = null
          ssl.keystore.type = JKS
          ssl.protocol = TLSv1.3
          ssl.provider = null
          ssl.secure.random.implementation = null
          ssl.trustmanager.algorithm = PKIX
          ssl.truststore.certificates = null
          ssl.truststore.location = null
          ssl.truststore.password = null
          ssl.truststore.type = JKS
          transaction.timeout.ms = 60000
          transactional.id = null
          value.serializer = class org.apache.kafka.common.serialization.StringSerializer
       (org.apache.kafka.clients.producer.ProducerConfig)
      [2023-05-02 19:50:24,714] INFO Kafka version: 6.1.1-ccs (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,715] INFO Kafka commitId: c209f70c6c2e52ae (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,715] INFO Kafka startTimeMs: 1683057024714 (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,753] INFO [Producer clientId=cdc_mysql_avro.local2-schemahistory] Cluster ID: LgjEAa6RQUmsbH92o0eZsw (org.apache.kafka.clients.Metadata)
      [2023-05-02 19:50:24,831] INFO Closing connection before starting schema recovery (io.debezium.connector.mysql.MySqlConnectorTask)
      [2023-05-02 19:50:24,836] INFO Connection gracefully closed (io.debezium.jdbc.JdbcConnection)
      [2023-05-02 19:50:24,837] INFO Connector started for the first time, database schema history recovery will not be executed (io.debezium.connector.mysql.MySqlConnectorTask)
      [2023-05-02 19:50:24,841] INFO ConsumerConfig values:
          allow.auto.create.topics = true
          auto.commit.interval.ms = 5000
          auto.offset.reset = earliest
          bootstrap.servers = [kafka:29092]
          check.crcs = true
          client.dns.lookup = use_all_dns_ips
          client.id = cdc_mysql_avro.local2-schemahistory
          client.rack =
          connections.max.idle.ms = 540000
          default.api.timeout.ms = 60000
          enable.auto.commit = false
          exclude.internal.topics = true
          fetch.max.bytes = 52428800
          fetch.max.wait.ms = 500
          fetch.min.bytes = 1
          group.id = cdc_mysql_avro.local2-schemahistory
          group.instance.id = null
          heartbeat.interval.ms = 3000
          interceptor.classes = []
          internal.leave.group.on.close = true
          internal.throw.on.fetch.stable.offset.unsupported = false
          isolation.level = read_uncommitted
          key.deserializer = class org.apache.kafka.common.serialization.StringDeserializer
          max.partition.fetch.bytes = 1048576
          max.poll.interval.ms = 300000
          max.poll.records = 500
          metadata.max.age.ms = 300000
          metric.reporters = []
          metrics.num.samples = 2
          metrics.recording.level = INFO
          metrics.sample.window.ms = 30000
          partition.assignment.strategy = [class org.apache.kafka.clients.consumer.RangeAssignor]
          receive.buffer.bytes = 65536
          reconnect.backoff.max.ms = 1000
          reconnect.backoff.ms = 50
          request.timeout.ms = 30000
          retry.backoff.ms = 100
          sasl.client.callback.handler.class = null
          sasl.jaas.config = null
          sasl.kerberos.kinit.cmd = /usr/bin/kinit
          sasl.kerberos.min.time.before.relogin = 60000
          sasl.kerberos.service.name = null
          sasl.kerberos.ticket.renew.jitter = 0.05
          sasl.kerberos.ticket.renew.window.factor = 0.8
          sasl.login.callback.handler.class = null
          sasl.login.class = null
          sasl.login.refresh.buffer.seconds = 300
          sasl.login.refresh.min.period.seconds = 60
          sasl.login.refresh.window.factor = 0.8
          sasl.login.refresh.window.jitter = 0.05
          sasl.mechanism = GSSAPI
          security.protocol = PLAINTEXT
          security.providers = null
          send.buffer.bytes = 131072
          session.timeout.ms = 10000
          socket.connection.setup.timeout.max.ms = 127000
          socket.connection.setup.timeout.ms = 10000
          ssl.cipher.suites = null
          ssl.enabled.protocols = [TLSv1.2, TLSv1.3]
          ssl.endpoint.identification.algorithm = https
          ssl.engine.factory.class = null
          ssl.key.password = null
          ssl.keymanager.algorithm = SunX509
          ssl.keystore.certificate.chain = null
          ssl.keystore.key = null
          ssl.keystore.location = null
          ssl.keystore.password = null
          ssl.keystore.type = JKS
          ssl.protocol = TLSv1.3
          ssl.provider = null
          ssl.secure.random.implementation = null
          ssl.trustmanager.algorithm = PKIX
          ssl.truststore.certificates = null
          ssl.truststore.location = null
          ssl.truststore.password = null
          ssl.truststore.type = JKS
          value.deserializer = class org.apache.kafka.common.serialization.StringDeserializer
       (org.apache.kafka.clients.consumer.ConsumerConfig)
      [2023-05-02 19:50:24,857] INFO Kafka version: 6.1.1-ccs (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,862] INFO Kafka commitId: c209f70c6c2e52ae (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,863] INFO Kafka startTimeMs: 1683057024857 (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,878] INFO [Consumer clientId=cdc_mysql_avro.local2-schemahistory, groupId=cdc_mysql_avro.local2-schemahistory] Cluster ID: LgjEAa6RQUmsbH92o0eZsw (org.apache.kafka.clients.Metadata)
      [2023-05-02 19:50:24,889] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:24,890] INFO Closing reporter org.apache.kafka.common.metrics.JmxReporter (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:24,890] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:24,905] INFO App info kafka.consumer for cdc_mysql_avro.local2-schemahistory unregistered (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,908] INFO AdminClientConfig values:
          bootstrap.servers = [kafka:29092]
          client.dns.lookup = use_all_dns_ips
          client.id = cdc_mysql_avro.local2-schemahistory
          connections.max.idle.ms = 300000
          default.api.timeout.ms = 60000
          metadata.max.age.ms = 300000
          metric.reporters = []
          metrics.num.samples = 2
          metrics.recording.level = INFO
          metrics.sample.window.ms = 30000
          receive.buffer.bytes = 65536
          reconnect.backoff.max.ms = 1000
          reconnect.backoff.ms = 50
          request.timeout.ms = 30000
          retries = 1
          retry.backoff.ms = 100
          sasl.client.callback.handler.class = null
          sasl.jaas.config = null
          sasl.kerberos.kinit.cmd = /usr/bin/kinit
          sasl.kerberos.min.time.before.relogin = 60000
          sasl.kerberos.service.name = null
          sasl.kerberos.ticket.renew.jitter = 0.05
          sasl.kerberos.ticket.renew.window.factor = 0.8
          sasl.login.callback.handler.class = null
          sasl.login.class = null
          sasl.login.refresh.buffer.seconds = 300
          sasl.login.refresh.min.period.seconds = 60
          sasl.login.refresh.window.factor = 0.8
          sasl.login.refresh.window.jitter = 0.05
          sasl.mechanism = GSSAPI
          security.protocol = PLAINTEXT
          security.providers = null
          send.buffer.bytes = 131072
          socket.connection.setup.timeout.max.ms = 127000
          socket.connection.setup.timeout.ms = 10000
          ssl.cipher.suites = null
          ssl.enabled.protocols = [TLSv1.2, TLSv1.3]
          ssl.endpoint.identification.algorithm = https
          ssl.engine.factory.class = null
          ssl.key.password = null
          ssl.keymanager.algorithm = SunX509
          ssl.keystore.certificate.chain = null
          ssl.keystore.key = null
          ssl.keystore.location = null
          ssl.keystore.password = null
          ssl.keystore.type = JKS
          ssl.protocol = TLSv1.3
          ssl.provider = null
          ssl.secure.random.implementation = null
          ssl.trustmanager.algorithm = PKIX
          ssl.truststore.certificates = null
          ssl.truststore.location = null
          ssl.truststore.password = null
          ssl.truststore.type = JKS
       (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,915] WARN The configuration 'value.serializer' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,915] WARN The configuration 'acks' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,915] WARN The configuration 'batch.size' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,915] WARN The configuration 'max.block.ms' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,916] WARN The configuration 'buffer.memory' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,916] WARN The configuration 'key.serializer' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,916] WARN The configuration 'linger.ms' was supplied but isn't a known config. (org.apache.kafka.clients.admin.AdminClientConfig)
      [2023-05-02 19:50:24,916] INFO Kafka version: 6.1.1-ccs (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,916] INFO Kafka commitId: c209f70c6c2e52ae (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:24,916] INFO Kafka startTimeMs: 1683057024916 (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:25,627] INFO Database schema history topic '(name=my_schema_history2, numPartitions=1, replicationFactor=default, replicasAssignments=null, configs={cleanup.policy=delete, retention.ms=9223372036854775807, retention.bytes=-1})' created (io.debezium.storage.kafka.history.KafkaSchemaHistory)
      [2023-05-02 19:50:25,629] INFO App info kafka.admin.client for cdc_mysql_avro.local2-schemahistory unregistered (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:50:25,654] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:25,655] INFO Closing reporter org.apache.kafka.common.metrics.JmxReporter (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:25,655] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:50:25,656] INFO Reconnecting after finishing schema recovery (io.debezium.connector.mysql.MySqlConnectorTask)
      [2023-05-02 19:50:25,797] INFO Requested thread factory for connector MySqlConnector, id = cdc_mysql_avro.local2 named = change-event-source-coordinator (io.debezium.util.Threads)
      [2023-05-02 19:50:25,806] INFO Creating thread debezium-mysqlconnector-cdc_mysql_avro.local2-change-event-source-coordinator (io.debezium.util.Threads)
      [2023-05-02 19:50:25,808] INFO Successfully started task (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:25,822] INFO Metrics registered (io.debezium.pipeline.ChangeEventSourceCoordinator)
      [2023-05-02 19:50:25,823] INFO Context created (io.debezium.pipeline.ChangeEventSourceCoordinator)
      [2023-05-02 19:50:25,847] INFO No previous offset has been found (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:25,847] INFO According to the connector configuration only schema will be snapshotted (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:25,849] INFO Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:25,854] INFO Snapshot step 2 - Determining captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:25,854] INFO Read list of available databases (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:25,909] INFO      list of available databases is: [information_schema, mysql, performance_schema, sys] (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:25,911] INFO Read list of available tables in each database (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:25,984] INFO     snapshot continuing with database(s): [information_schema, performance_schema, mysql, sys] (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:26,002] INFO Snapshot step 3 - Locking captured tables [] (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:26,045] INFO Flush and obtain global read lock to prevent writes to database (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:26,051] INFO Snapshot step 4 - Determining snapshot offset (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:26,074] INFO Read binlog position of MySQL primary server (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:26,113] INFO      using binlog 'mysql-bin.000003' at position '196' and gtid 'fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5' (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:26,114] INFO Snapshot step 5 - Reading structure of captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:26,114] INFO All eligible tables schema should be captured, capturing: [] (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:28,950] INFO Snapshot step 6 - Persisting schema history (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:29,187] INFO Already applied 1 database changes (io.debezium.relational.history.SchemaHistoryMetrics)
      [2023-05-02 19:50:29,208] INFO Releasing global read lock to enable MySQL writes (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:29,210] INFO Writes to MySQL tables prevented for a total of 00:00:03.161 (io.debezium.connector.mysql.MySqlSnapshotChangeEventSource)
      [2023-05-02 19:50:29,211] INFO Snapshot step 7 - Skipping snapshotting of data (io.debezium.relational.RelationalSnapshotChangeEventSource)
      [2023-05-02 19:50:29,218] INFO Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource)
      [2023-05-02 19:50:29,218] INFO Snapshot completed (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource)
      [2023-05-02 19:50:29,496] INFO 1 records sent during previous 00:00:05.844, last recorded offset of {server=cdc_mysql_avro.local2} partition is {ts_sec=1683057026, file=mysql-bin.000003, pos=196, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5} (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:50:29,696] INFO Snapshot ended with SnapshotResult [status=COMPLETED, offset=MySqlOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.mysql.Source:STRUCT}, sourceInfo=SourceInfo [currentGtid=null, currentBinlogFilename=mysql-bin.000003, currentBinlogPosition=196, currentRowNumber=0, serverId=0, sourceTime=2023-05-02T19:50:28.956Z, threadId=-1, currentQuery=null, tableIds=[null], databaseName=], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5, currentGtidSet=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5, restartBinlogFilename=mysql-bin.000003, restartBinlogPosition=196, restartRowsToSkip=0, restartEventsToSkip=0, currentEventLengthInBytes=0, inTransaction=false, transactionId=null, incrementalSnapshotContext =IncrementalSnapshotContext [windowOpened=false, chunkEndPosition=null, dataCollectionsToSnapshot=[], lastEventKeySent=null, maximumKey=null]]] (io.debezium.pipeline.ChangeEventSourceCoordinator)
      [2023-05-02 19:50:29,697] WARN After applying the include/exclude list filters, no changes will be captured. Please check your configuration! (io.debezium.relational.RelationalDatabaseSchema)
      [2023-05-02 19:50:29,725] INFO Requested thread factory for connector MySqlConnector, id = cdc_mysql_avro.local2 named = binlog-client (io.debezium.util.Threads)
      [2023-05-02 19:50:29,761] INFO Starting streaming (io.debezium.pipeline.ChangeEventSourceCoordinator)
      [2023-05-02 19:50:29,762] WARN After applying the include/exclude list filters, no changes will be captured. Please check your configuration! (io.debezium.relational.RelationalDatabaseSchema)
      [2023-05-02 19:50:29,910] INFO GTID set purged on server:  (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,910] INFO Attempting to generate a filtered GTID set (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,910] INFO GTID set from previous recorded offset: fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,911] INFO GTID set available on server: fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,911] INFO Using first available positions for new GTID channels (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,911] INFO Relevant GTID set available on server: fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,955] INFO Final merged GTID set to use when connecting to MySQL: fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,955] INFO Registering binlog reader with GTID set: fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,962] INFO Skip 0 events on streaming start (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,962] INFO Skip 0 rows on streaming start (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:29,978] INFO Creating thread debezium-mysqlconnector-cdc_mysql_avro.local2-binlog-client (io.debezium.util.Threads)
      [2023-05-02 19:50:30,015] INFO Creating thread debezium-mysqlconnector-cdc_mysql_avro.local2-binlog-client (io.debezium.util.Threads)
      May 02, 2023 7:50:30 PM com.github.shyiko.mysql.binlog.BinaryLogClient connect
      INFO: Connected to taxify-binlog-db:3306 at fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5 (sid:184055, cid:11)
      [2023-05-02 19:50:30,118] INFO Connected to MySQL binlog at taxify-binlog-db:3306, starting at MySqlOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.mysql.Source:STRUCT}, sourceInfo=SourceInfo [currentGtid=null, currentBinlogFilename=mysql-bin.000003, currentBinlogPosition=196, currentRowNumber=0, serverId=0, sourceTime=2023-05-02T19:50:28.956Z, threadId=-1, currentQuery=null, tableIds=[null], databaseName=], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5, currentGtidSet=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5, restartBinlogFilename=mysql-bin.000003, restartBinlogPosition=196, 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)
      [2023-05-02 19:50:30,120] INFO Waiting for keepalive thread to start (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:30,133] INFO Creating thread debezium-mysqlconnector-cdc_mysql_avro.local2-binlog-client (io.debezium.util.Threads)
      [2023-05-02 19:50:30,223] INFO Keepalive thread is running (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:50:32,471] WARN [Producer clientId=connector-producer-source-mysql-local2-0] Error while fetching metadata with correlation id 3 : {cdc_mysql_avro.local2=LEADER_NOT_AVAILABLE} (org.apache.kafka.clients.NetworkClient)
      [2023-05-02 19:50:32,676] WARN [Producer clientId=connector-producer-source-mysql-local2-0] Error while fetching metadata with correlation id 4 : {cdc_mysql_avro.local2=LEADER_NOT_AVAILABLE} (org.apache.kafka.clients.NetworkClient)
      [2023-05-02 19:50:47,855] INFO Already applied 2 database changes (io.debezium.relational.history.SchemaHistoryMetrics)
      [2023-05-02 19:50:47,868] INFO 1 records sent during previous 00:00:18.371, last recorded offset of {server=cdc_mysql_avro.local2} partition is {transaction_id=null, ts_sec=1683057047, file=mysql-bin.000003, pos=378, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-5, server_id=223344} (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:51:22,594] INFO Already applied 3 database changes (io.debezium.relational.history.SchemaHistoryMetrics)
      [2023-05-02 19:51:22,637] INFO 1 records sent during previous 00:00:34.77, last recorded offset of {server=cdc_mysql_avro.local2} partition is {transaction_id=null, ts_sec=1683057081, file=mysql-bin.000003, pos=1930, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-6, server_id=223344} (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:51:24,014] INFO WorkerSourceTask{id=source-mysql-local2-0} Committing offsets (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:51:24,017] INFO WorkerSourceTask{id=source-mysql-local2-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:51:24,121] INFO WorkerSourceTask{id=source-mysql-local2-0} Finished commitOffsets successfully in 95 ms (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:51:49,823] INFO 1 records sent during previous 00:00:27.186, last recorded offset of {server=cdc_mysql_avro.local2} partition is {transaction_id=null, ts_sec=1683057109, file=mysql-bin.000003, pos=2009, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-7, row=1, server_id=223344, event=2} (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:51:50,199] WARN [Producer clientId=connector-producer-source-mysql-local2-0] Error while fetching metadata with correlation id 9 : {cdc_mysql_avro.local2.db1.t1=LEADER_NOT_AVAILABLE} (org.apache.kafka.clients.NetworkClient)
      [2023-05-02 19:52:23,021] INFO Already applied 4 database changes (io.debezium.relational.history.SchemaHistoryMetrics)
      [2023-05-02 19:52:24,123] INFO WorkerSourceTask{id=source-mysql-local2-0} Committing offsets (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:52:24,123] INFO WorkerSourceTask{id=source-mysql-local2-0} flushing 0 outstanding messages for offset commit (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:52:24,143] INFO WorkerSourceTask{id=source-mysql-local2-0} Finished commitOffsets successfully in 20 ms (org.apache.kafka.connect.runtime.WorkerSourceTask)
      [2023-05-02 19:53:04,816] ERROR Error during binlog processing. Last offset stored = {transaction_id=null, ts_sec=1683057142, file=mysql-bin.000003, pos=2703, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-9, server_id=223344, event=1}, binlog reader near position = mysql-bin.000003/2785 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:53:04,843] ERROR Producer failure (io.debezium.pipeline.ErrorHandler)
      io.debezium.DebeziumException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1236)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onEventDeserializationFailure(MySqlStreamingChangeEventSource.java:1289)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1064)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:631)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:932)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:341)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeTableMapEventData(EventDeserializer.java:313)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.nextEvent(EventDeserializer.java:237)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$1.nextEvent(MySqlStreamingChangeEventSource.java:232)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1051)
          ... 3 more
      Caused by: java.io.IOException: Unsupported table metadata field type 0
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventMetadataDeserializer.deserialize(TableMapEventMetadataDeserializer.java:52)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:47)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:27)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:335)
          ... 7 more
      [2023-05-02 19:53:04,860] WARN Retry 1 of unlimited retries will be attempted (io.debezium.pipeline.ErrorHandler)
      [2023-05-02 19:53:04,861] ERROR Error during binlog processing. Last offset stored = {transaction_id=null, ts_sec=1683057142, file=mysql-bin.000003, pos=2703, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-9, server_id=223344, event=1}, binlog reader near position = mysql-bin.000003/2785 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:53:04,862] ERROR Producer failure (io.debezium.pipeline.ErrorHandler)
      io.debezium.DebeziumException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=EXT_WRITE_ROWS, serverId=223344, headerLength=19, dataLength=116, nextPosition=3064, flags=0}
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1236)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onEventDeserializationFailure(MySqlStreamingChangeEventSource.java:1289)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1064)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:631)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:932)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=EXT_WRITE_ROWS, serverId=223344, headerLength=19, dataLength=116, nextPosition=3064, flags=0}
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:341)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.nextEvent(EventDeserializer.java:244)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$1.nextEvent(MySqlStreamingChangeEventSource.java:232)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1051)
          ... 3 more
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.MissingTableMapEventException: No TableMapEventData has been found for table id:117. Usually that means that you have started reading binary log 'within the logical event group' (e.g. from WRITE_ROWS and not proceeding TABLE_MAP
          at com.github.shyiko.mysql.binlog.event.deserialization.AbstractRowsEventDataDeserializer.deserializeRow(AbstractRowsEventDataDeserializer.java:109)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserializeRows(WriteRowsEventDataDeserializer.java:64)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserialize(WriteRowsEventDataDeserializer.java:56)
          at com.github.shyiko.mysql.binlog.event.deserialization.WriteRowsEventDataDeserializer.deserialize(WriteRowsEventDataDeserializer.java:32)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:335)
          ... 6 more
      [2023-05-02 19:53:05,357] WARN Going to restart connector after 10 sec. after a retriable exception (io.debezium.connector.common.BaseSourceTask)
      May 02, 2023 7:53:05 PM com.github.shyiko.mysql.binlog.BinaryLogClient$5 run
      INFO: threadExecutor is shut down, terminating keepalive thread
      [2023-05-02 19:53:05,426] INFO Finished streaming (io.debezium.pipeline.ChangeEventSourceCoordinator)
      [2023-05-02 19:53:05,477] INFO Stopped reading binlog after 0 events, last recorded offset: {transaction_id=null, ts_sec=1683057142, file=mysql-bin.000003, pos=3095, gtids=fa44e475-e921-11ed-a0e5-0242ac1b0003:1-10, server_id=223344, event=1} (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2023-05-02 19:53:05,477] INFO Connection gracefully closed (io.debezium.jdbc.JdbcConnection)
      [2023-05-02 19:53:05,484] INFO [Producer clientId=cdc_mysql_avro.local2-schemahistory] Closing the Kafka producer with timeoutMillis = 30000 ms. (org.apache.kafka.clients.producer.KafkaProducer)
      [2023-05-02 19:53:05,511] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:53:05,511] INFO Closing reporter org.apache.kafka.common.metrics.JmxReporter (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:53:05,511] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics)
      [2023-05-02 19:53:05,518] INFO App info kafka.producer for cdc_mysql_avro.local2-schemahistory unregistered (org.apache.kafka.common.utils.AppInfoParser)
      [2023-05-02 19:53:05,519] WARN WorkerSourceTask{id=source-mysql-local2-0} failed to poll records from SourceTask. Will retry operation. (org.apache.kafka.connect.runtime.WorkerSourceTask)
      org.apache.kafka.connect.errors.RetriableException: An exception occurred in the change event producer. This connector will be restarted.
          at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:68)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onEventDeserializationFailure(MySqlStreamingChangeEventSource.java:1289)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1064)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:631)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:932)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: io.debezium.DebeziumException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1236)
          ... 5 more
      Caused by: com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException: Failed to deserialize data of EventHeaderV4{timestamp=1683057184000, eventType=TABLE_MAP, serverId=223344, headerLength=19, dataLength=125, nextPosition=2929, flags=0}
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:341)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeTableMapEventData(EventDeserializer.java:313)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.nextEvent(EventDeserializer.java:237)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$1.nextEvent(MySqlStreamingChangeEventSource.java:232)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1051)
          ... 3 more
      Caused by: java.io.IOException: Unsupported table metadata field type 0
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventMetadataDeserializer.deserialize(TableMapEventMetadataDeserializer.java:52)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:47)
          at com.github.shyiko.mysql.binlog.event.deserialization.TableMapEventDataDeserializer.deserialize(TableMapEventDataDeserializer.java:27)
          at com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer.deserializeEventData(EventDeserializer.java:335)
          ... 7 more
      [2023-05-02 19:53:05,522] INFO Awaiting end of restart backoff period after a retriable error (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:53:07,523] INFO Awaiting end of restart backoff period after a retriable error (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:53:09,526] INFO Awaiting end of restart backoff period after a retriable error (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:53:11,530] INFO Awaiting end of restart backoff period after a retriable error (io.debezium.connector.common.BaseSourceTask)
      [2023-05-02 19:53:13,534] INFO Awaiting end of restart backoff period after a retriable error (io.debezium.connector.common.BaseSourceTask) 

       

      Attachments

        Activity

          People

            Unassigned Unassigned
            kewei.shang kewei shang
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: