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

Primary Key Update/ Snapshot Race Condition

XMLWordPrintable

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

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      connector: mysql

      version: 2.6.1.Final

      What is the connector configuration?

          "connector.class": "io.debezium.connector.mysql.MySqlConnector",
          "snapshot.locking.mode": "minimal_percona",
          "signal.enabled.channels": "kafka",
          "schema.history.internal.kafka.recovery.attempts": "10",
          "snapshot.lock.timeout.ms": "12000000",
          "transforms.redactBefore.negate": "true",
          "min.row.count.to.stream.results": "0",
          "snapshot.mode": "schema_only",
          "incremental.snapshot.chunk.size": "1000",    
          "tombstones.on.delete": "false",   
          "transforms.topicPerShard.type": "org.apache.kafka.connect.transforms.RegexRouter",
          "database.allowPublicKeyRetrieval": "true",    
          "notification.enabled.channels": "sink",    
          "schema.name.adjustment.mode": "avro",    
          "include.query": "true",    
          "signal.kafka.poll.timeout.ms": "1500",
          "producer.override.buffer.memory": "67108864",
          "signal.consumer.fetch.max.wait.ms": "1500",    
          "producer.override.linger.ms": "5",    
          "database.history.kafka.recovery.poll.interval.ms": "15000",
          "poll.interval.ms": "5",    
          "key.converter": "io.confluent.connect.avro.AvroConverter",    
          "heartbeat.interval.ms": "60000",        
          "max.queue.size": "16384",    
          "tasks.max": "1",
          "producer.override.batch.size": "262144",    
          "database.history.kafka.recovery.attempts": "10",
          "offset.flush.timeout.ms": "4000",
          "read.only": "true",

      What is the captured database version and mode of deployment?

      mysql version 8

      What behavior do you expect?

      The source.table name to be the same across the primary key update c and d events.

      What behavior do you see?

      On a primary key update we witnessed the normal behavior of a delete event followed by a create event. However, the source.table name was mutated across records, i.e. the create event had a source.table value that was incorrect. 

      Our internal investigation uncovered that there was a incremental snapshot running on the same source.table table that was seen on the create event. This has lead us to believe that some non-threadsafe code mutated the object across the binlog and snapshot processing. This then lead to the table name of the snapshot table being used on the delete event debezium creates from the primary key update event on the binlog. 

      Looking through the Debezium source, we potentially narrowed down the issue to RelationalChangeRecordEmitter#emitUpdateAsPrimaryKeyChangeRecordwhere the getOffset().getSourceInfo() potentially being mutated in another thread between the tableSchema.getEnvelopeSchema().delete and tableSchema.getEnvelopeSchema().create invocations.

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

      Haven't tested on latest. 

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

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

      The logs can be provided if required, though there is no data there outside of log statements around the incremental snapshot being submitted/processed.

      How to reproduce the issue using our tutorial deployment?

      <Your answer>

      Feature request or enhancement

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

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

      <Your answer>

      Implementation ideas (optional)

      <Your answer>

            vjuranek@redhat.com Vojtech Juranek
            jtanza John Tanza (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

              Created:
              Updated:
              Resolved: