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

GTIDs are not updated at the restart of a failed initial snapshot

    XMLWordPrintable

Details

    • False
    • None
    • False

    Description

      Hello,

      I'm using latest debezium version 1.9.0.Final on a Debian 10 OS with openjdk 11.

      I'm currently dealing with an issue during a really long initial snapshot.
      If the snapshot fails, a task restart will restart the whole process as described here https://issues.redhat.com/browse/DBZ-5008.

      However, Debezium will not update the status topic to use the current server GtidSet while it should (it starts a new transaction to dump the whole set). It will use the GtidSet taken from the first snapshot try.

      As such, if some GTID are not available on the server anymore, binlog streaming will fail at the end of the last snapshot.

      [2022-04-21 07:47:30,344] INFO Found previous partition offset MySqlPartition [sourcePartition={server=paiement}]: {file=xxxx-node-2-log-bin.022188, pos=1616964, gtids=xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246, snapshot=true} (io.debezium.connector.common.BaseSourceTask)[2022-04-21 11:28:31,804] INFO Snapshot ended with SnapshotResult [status=COMPLETED, offset=MySqlOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.mysql.Source:STRUCT}, sourceInfo=SourceInfo [currentGtid=null, currentBinlogFilename=xxxx-node-2-log-bin.022188, currentBinlogPosition=1616964, currentRowNumber=0, serverId=0, sourceTime=2022-04-21T11:28:31.802Z, threadId=-1, currentQuery=null, tableIds=[paiement.withdrawFrCA], databaseName=paiement], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246, currentGtidSet=xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246, restartBinlogFilename=xxxx-node-2-log-bin.022188, restartBinlogPosition=1616964, 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)
      
      
      
      [2022-04-21 11:28:31,833] INFO GTID set purged on server: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2015052434 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,833] INFO Attempting to generate a filtered GTID set (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,833] INFO GTID set from previous recorded offset: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,833] INFO GTID set available on server: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2021766258 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,833] INFO Using first available positions for new GTID channels (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,833] INFO Relevant GTID set available on server: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2021766258 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,834] INFO Final merged GTID set to use when connecting to MySQL: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      [2022-04-21 11:28:31,834] INFO Registering binlog reader with GTID set: xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246 (io.debezium.connector.mysql.MySqlStreamingChangeEventSource)
      
      
      
      [2022-04-21 11:28:31,840] INFO Connected to MySQL binlog at X.X.X.X:3306, starting at MySqlOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.mysql.Source:STRUCT}, sourceInfo=SourceInfo [currentGtid=null, currentBinlogFilename=xxxx-node-2-log-bin.022188, currentBinlogPosition=1616964, currentRowNumber=0, serverId=0, sourceTime=2022-04-21T11:28:31.802Z, threadId=-1, currentQuery=null, tableIds=[xxxxx.yyyy], databaseName=paiement], snapshotCompleted=true, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], restartGtidSet=xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246, currentGtidSet=xxxyyyzz-641f-11ea-b33f-001e67ec98aa:1-2008047246, restartBinlogFilename=xxxx-node-2-log-bin.022188, restartBinlogPosition=1616964, 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 
      
      
      
      [2022-04-21 11:28:32,909] ERROR WorkerSourceTask{id=paiement-0} Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask)
      org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
          at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:50)
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource$ReaderThreadLifecycleListener.onCommunicationFailure(MySqlStreamingChangeEventSource.java:1224)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:980)
          at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:599)
          at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:857)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: io.debezium.DebeziumException: Cannot replicate because the master purged required binary logs. Replicate the missing transactions from elsewhere, or provision a new slave from backup. Consider increasing the master's binary log expiration period. The GTID sets and the missing purged transactions are too long to print in this message. For more information, please see the master's error log or the manual for GTID_SUBTRACT Error code: 1236; SQLSTATE: HY000.
          at io.debezium.connector.mysql.MySqlStreamingChangeEventSource.wrap(MySqlStreamingChangeEventSource.java:1179)
          ... 5 more
      Caused by: com.github.shyiko.mysql.binlog.network.ServerException: Cannot replicate because the master purged required binary logs. Replicate the missing transactions from elsewhere, or provision a new slave from backup. Consider increasing the master's binary log expiration period. The GTID sets and the missing purged transactions are too long to print in this message. For more information, please see the master's error log or the manual for GTID_SUBTRACT
          at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:944)
          ... 3 more

       

      Current workaround :

      • Delete the connector
      • Send a tombstone on the connect status topic on the correct key
      • Recreate the connector

      Attachments

        Activity

          People

            Unassigned Unassigned
            fuyar UYAR Farid (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated: