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

MySQL active-passive: brief data loss on failover when Debezium encounters new GTID channel

    • Icon: Bug Bug
    • Resolution: Done
    • Icon: Critical Critical
    • 0.9.0.Beta2
    • 0.8.3.Final
    • mysql-connector
    • None

      Lets say we have two mysql servers in standard active-passive high availability setup. If current master node fails, automation will promote passive instance to new master and it continues to serve live traffic. And debezium is connecting to master node as well.

      Starting point:
      Server A (current master)
      uuid: abc
      gtids: abc:1-100

      Server B (slave)
      uuid: dfg
      gtid: abc:1-100 (replating from master)

      Debezium is connecting to master also, so it has
      gtids: abc:1-100

      Now assume master node fails, failover is triggered

      Server B (automation promotes it to new master)
      uuid: dfg,
      gtids: abc:1-100, dfg: 1-20

      Server A (becomes slave, starts replication from B)
      uuid: abc
      gtids: abc:1-100, dfg: 1-20

      Debezium after job restart:
      gtids: abc:1-100, dfg:1-20,

      Debezium gets connection reset error, then on job restart it successfully connects to new master (Server B), finds new gtid channel (dfg) and merges it to existing offsets and connects.

      Works, BUT! There is a timing issue.

      When encountering new gtid debezium starts reading it from mysql server latest gtid_executed position. So in case when mysql servers failover happens faster than debezium job failure detection and restart, the live data arriving to new master with new gtid channel (dfg in our example) is never processed in debezium. In our infra it can be several minutes of data lost as with large schemas debezium startup takes some time.

      What do you think about option to specify what should debezium do when encountering new gtid - take the latest executed position and continue from there or take earlies available value on server. Default could remain "latest", but in our case "earliest" would solve our problem with lost data changes on failover. Earliest could be gtid_purged channel value or if nothing purged then from position 1.

            [DBZ-923] MySQL active-passive: brief data loss on failover when Debezium encounters new GTID channel

            Released

            Jiri Pechanec added a comment - Released

            Created pull request for docs update https://github.com/debezium/debezium.github.io/pull/233

            Eero Koplimets (Inactive) added a comment - Created pull request for docs update https://github.com/debezium/debezium.github.io/pull/233

            Thanks for reporting back, pimpelsang. Btw. there's still the doc update for the new option missing. Could you file a PR for adding it to the connector docs? Thanks!

            Gunnar Morling added a comment - Thanks for reporting back, pimpelsang . Btw. there's still the doc update for the new option missing. Could you file a PR for adding it to the connector docs? Thanks!

            just to follow up this nightly has been running now in our env 4 days without problems.

            Eero Koplimets (Inactive) added a comment - just to follow up this nightly has been running now in our env 4 days without problems.

            sure, will do

            Eero Koplimets (Inactive) added a comment - sure, will do

            pimpelsang, I've added one more commit with some clean-up. Perhaps you can build the connector from source (or wait for today's nightly build) and give it another test run in your environment?

            Gunnar Morling added a comment - pimpelsang , I've added one more commit with some clean-up. Perhaps you can build the connector from source (or wait for today's nightly build) and give it another test run in your environment?

            The code change has been merged, leaving the issue open until there's a docs PR, too.

            Gunnar Morling added a comment - The code change has been merged, leaving the issue open until there's a docs PR, too.

            Gunnar Morling added a comment - - edited

            Thanks a lot for the thorough analysis and the PR, pimpelsang! There's one potential issue I see with the proposed EARLIEST mode, and that'd be master-master set-ups. In that case the connector would already have streamed some "dfg" changes while reading from server A, so there'd be duplicated events. I reckon there's no way to avoid it, though, and after all, Debezium generally works with "at least once" semantics, so seems acceptable.

            So thinking more about this, this should be fine actually. In this case the connector would have committed the offsets while reading from A, so there shouldn't be any more duplication than the always to be expected one of events emitted after the last offset commit.

            Gunnar Morling added a comment - - edited Thanks a lot for the thorough analysis and the PR, pimpelsang ! There's one potential issue I see with the proposed EARLIEST mode, and that'd be master-master set-ups. In that case the connector would already have streamed some "dfg" changes while reading from server A, so there'd be duplicated events. I reckon there's no way to avoid it, though, and after all, Debezium generally works with "at least once" semantics, so seems acceptable. So thinking more about this, this should be fine actually. In this case the connector would have committed the offsets while reading from A, so there shouldn't be any more duplication than the always to be expected one of events emitted after the last offset commit.

            Mostly this applies to failovers as when first time starting up and creating snapshot the current database state gets used. But when you already have offsets, then it could be that restored mysql server has new gtid channels and then data loss. There sure will be errors when that lost events part contained alters.

            Eero Koplimets (Inactive) added a comment - Mostly this applies to failovers as when first time starting up and creating snapshot the current database state gets used. But when you already have offsets, then it could be that restored mysql server has new gtid channels and then data loss. There sure will be errors when that lost events part contained alters.

            Excellent analysis! A pull request is definitely welcome. Is it only a problem with failover? Cannot something like this happen when Debezium fails to start with unfortunate timing?

            Jiri Pechanec added a comment - Excellent analysis! A pull request is definitely welcome. Is it only a problem with failover? Cannot something like this happen when Debezium fails to start with unfortunate timing?

              Unassigned Unassigned
              pimpelsang Eero Koplimets (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: