Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-14402 Transactions Server Side Defense
  3. KAFKA-18019

Convert INVALID_PRODUCER_ID_MAPPING from abortable error to fatal error

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 4.0.0
    • None
    • None

    Description

      Since we bump epoch on abort, we no longer need to call InitProducerId to fence requests. InitProducerId will only be called when the producer starts up to fence a previous instance. 

      With this change, some other calls to InitProducerId were inspected including the call after receiving an InvalidPidMappingException. This exception was changed to abortable as part of KIP-360: Improve reliability of idempotent/transactional producer. However, this change means that we can violate EOS guarantees. As an example:

      Consider an application that is copying data from one partition to another

      • Application instance A processes to offset 4
      • Application instance B comes up and fences application instance A
      • Application instance B processes to offset 5
      • Application instances A and B are idle for transaction.id.expiration.ms, transaction id expires on server
      • Application instance A attempts to process offset 5 (since in its view, that is next) – if we recover from invalid pid mapping, we can duplicate this processing

      Thus, INVALID_PID_MAPPING should be fatal to the producer. 

      This is consistent with KIP-1050: Consistent error handling for Transactions where errors that are fatal to the producer are in the "application recoverable" category. This is a grouping that indicates to the client that the producer needs to restart and recovery on the application side is necessary. KIP-1050 is approved so we are consistent with that decision.

      Attachments

        Issue Links

          Activity

            People

              rreddy22 Ritika Reddy
              rreddy22 Ritika Reddy
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: