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

MySQL GTID failover does not work with filtered GTID sets

    XMLWordPrintable

Details

    • Bug
    • Resolution: Done
    • Blocker
    • 0.4
    • 0.3.6
    • mysql-connector
    • None

    Description

      We run our Debezium off of a MySQL cluster with HA proxy in front of it. (We're using GTIDs.) The topology is like this:

      MySQL1 \
              -- HA Proxy -- Debezium
      MySQL2 /
      

      We have HA proxy pointing to just MySQL 1. At some point, we do a fail over by switching HA proxy to point to MySQL2. When we do this, we see very strange behavior, where Debezium appears to start up, and then just sit there. No messages come through.

      With DEBUG enabled, we can see that Debezium is skipping all messages, INCLUDING messages that we believe it should be processing:

      Feb 06 13:23:17 dbzconnector01 kafkaconnect_logs:  [2017-02-06 21:23:07,379] DEBUG Skipping insert row event: Event{header=EventHeaderV4{timestamp=1486416187000, eventType=EXT_WRITE_ROWS, serverId=2822443076, headerLength=19, dataLength=1255, nextPosition=442649556, flags=0}, data=WriteRowsEventData{tableId=110, includedColumns={0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, 

      Furthermore, we see that Debezium appears to be skipping all DML when starting up, again including DML that it should be applying:

      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,076] DEBUG Skipping: SET character_set_server=latin1, collation_server=latin1_swedish_ci; (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,086] DEBUG Skipping: DROP TABLE IF EXISTS docs.schema_version (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,087] DEBUG Skipping: DROP TABLE IF EXISTS docs.table1 (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,087] DEBUG Skipping: DROP TABLE IF EXISTS docs.table2 (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,088] DEBUG Skipping: DROP DATABASE IF EXISTS docs (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,089] DEBUG Skipping: CREATE DATABASE docs (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,090] DEBUG Skipping: USE docs (io.debezium.relational.history.KafkaDatabaseHistory)
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,091] DEBUG Skipping: CREATE TABLE `oauth2_token` (
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,092] DEBUG Skipping: CREATE TABLE `table1` (
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,093] DEBUG Skipping: CREATE TABLE `table2` (
      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs:  [2017-02-06 22:24:07,094] DEBUG Skipping: SET character_set_server=latin1, collation_server=latin1_swedish_ci; (io.debezium.relational.history.KafkaDatabaseHistory)
      

      These tables should be included. When we dump the DB history table, we see rows as expected:

      {
        "source" : {
          "server" : "db.debezium.service"
        },
        "position" : {
          "file" : "mysql-bin.000008",
          "pos" : 380941551,
          "gtids" : "01261278-6ade-11e6-b36a-42010af00790:1-378422946,4d1a4918-44ba-11e6-bf12-42010af0040b:1-11002284,716ec46f-d522-11e5-bb56-0242ac110004:1-34673215,96c2072e-e428-11e6-9590-42010a28002d:1-3,c627b2bc-9647-11e6-a886-42010af0044a:1-9541144",
          "snapshot" : true
        },
        "ddl" : "DROP TABLE IF EXISTS docs.schema_version"
      }
      

      Everything looks fine, except the GTIDs don't quite match the GTIDs shown here:

      Feb 06 14:24:10 dbzconnector01 kafkaconnect_logs: [2017-02-06 22:24:06,725] DEBUG Recovering DDL history for source partition {server=db.debezium.service} and offset {file=mysql-bin.000016, pos=645115324, gtids=01261278-6ade-11e6-b36a-42010af00790:1-400944168,30efb117-e42a-11e6-ba9e-42010a28002e:1-9,4d1a4918-44ba-11e6-bf12-42010af0040b:1-11604379,621dc2f6-803b-11e6-acc1-42010af000a4:1-7963838,716ec46f-d522-11e5-bb56-0242ac110004:1-35850702,c627b2bc-9647-11e6-a886-42010af0044a:1-10426868,d079cbb3-750f-11e6-954e-42010af00c28:1-11544291:11544293-11885648, row=1, event=2} (io.debezium.relational.history.KafkaDatabaseHistory)
      

      Note how the DB history has 96c2072e-e428-11e6-9590-42010a28002d in the GTID, but the recovery log line does not. Likewise, the recovery log line has 30efb117-e42a-11e6-ba9e-42010a28002e, but the DB history line does not. These are the UUIDs of MySQL 1 and MySQL 2.

      We are applying filtered GTID sets as described in DBZ-143. I believe that the filtered GTIDs are being applied to the recovery portion, but not to the messages going into the DB history. I think that this leads to DMLs getting skipped when they should be being applied.

      Attachments

        Activity

          People

            rhauch Randall Hauch (Inactive)
            criccomini Chris Riccomini (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: