Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-4815 Idempotent/transactional Producer (KIP-98)
  3. KAFKA-5427

Transactional producer cannot find coordinator when trying to abort transaction after error

    XMLWordPrintableJSON

Details

    Description

      It can happen that we receive an abortable error while we are already aborting a transaction. In this case, we have an EndTxnRequest queued for sending when we transition to ABORTABLE_ERROR. It could be that we need to find the coordinator before sending this EndTxnRequest. The problem is that we will fail even the FindCoordinatorRequest because we are in an error state. This causes the following endless loop:

      [2017-06-10 19:29:33,436] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=FindCoordinatorRequest, coordinatorKey=my-fi
      rst-transactional-id, coordinatorType=TRANSACTION) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,436] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=EndTxnRequest, transactionalId=my-first-tran
      sactional-id, producerId=1000, producerEpoch=0, result=ABORT) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,536] TRACE [TransactionalId my-first-transactional-id] Not sending transactional request (type=FindCoordinatorRequest, coordinatorKey=my-
      first-transactional-id, coordinatorType=TRANSACTION) because we are in an error state (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,637] TRACE [TransactionalId my-first-transactional-id] Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, producerId
      =1000, producerEpoch=0, result=ABORT) dequeued for sending (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,637] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=FindCoordinatorRequest, coordinatorKey=my-fi
      rst-transactional-id, coordinatorType=TRANSACTION) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,637] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=EndTxnRequest, transactionalId=my-first-tran
      sactional-id, producerId=1000, producerEpoch=0, result=ABORT) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,737] TRACE [TransactionalId my-first-transactional-id] Not sending transactional request (type=FindCoordinatorRequest, coordinatorKey=my-
      first-transactional-id, coordinatorType=TRANSACTION) because we are in an error state (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,837] TRACE [TransactionalId my-first-transactional-id] Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, producerId
      =1000, producerEpoch=0, result=ABORT) dequeued for sending (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,838] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=FindCoordinatorRequest, coordinatorKey=my-fi
      rst-transactional-id, coordinatorType=TRANSACTION) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,838] DEBUG [TransactionalId my-first-transactional-id] Enqueuing transactional request (type=EndTxnRequest, transactionalId=my-first-tran
      sactional-id, producerId=1000, producerEpoch=0, result=ABORT) (org.apache.kafka.clients.producer.internals.TransactionManager)
      [2017-06-10 19:29:33,938] TRACE [TransactionalId my-first-transactional-id] Not sending transactional request (type=FindCoordinatorRequest, coordinatorKey=my-
      first-transactional-id, coordinatorType=TRANSACTION) because we are in an error state (org.apache.kafka.clients.producer.internals.TransactionManager)
      

      A couple suggested improvements:

      1. We should allow FindCoordinator requests regardless of the transaction state.
      2. It is a bit confusing that we allow EndTxnRequest to be sent in both the ABORTABLE_ERROR and the ABORTING_TRANSACTION states. Perhaps we should only allow EndTxnRequest to be sent in ABORTING_TRANSACTION. If we hit an abortable error and we are already aborting, then we should just stay in ABORTING_TRANSACTION and perhaps log a warning.

      Attachments

        Issue Links

          Activity

            People

              hachikuji Jason Gustafson
              hachikuji Jason Gustafson
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: