Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-13267

InvalidPidMappingException: The producer attempted to use a producer id which is not currently assigned to its transactional id

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.8.0
    • None
    • streams
    • None

    Description

      We're using Confluent Cloud and Kafka Streams 2.8.0 and we've seen these errors pop up in apps using EOS:

      InvalidPidMappingException: The producer attempted to use a producer id which is not currently assigned to its transactional id
      

      Full stack trace:

      Error encountered sending record to topic ola-update-1 for task 4_7 due to: org.apache.kafka.common.errors.InvalidPidMappingException: The producer attempted to use a producer id which is not currently assigned to its transactional id. Exception handler choose to FAIL the processing, no more records would be sent.
      RecordCollectorImpl.java  226  recordSendError(...)
      RecordCollectorImpl.java:226:in `recordSendError'
      RecordCollectorImpl.java  196  lambda$send$0(...)
      RecordCollectorImpl.java:196:in `lambda$send$0'
      KafkaProducer.java  1365  onCompletion(...)
      KafkaProducer.java:1365:in `onCompletion'
      ProducerBatch.java  231  completeFutureAndFireCallbacks(...)
      ProducerBatch.java:231:in `completeFutureAndFireCallbacks'
      ProducerBatch.java  159  abort(...)
      ProducerBatch.java:159:in `abort'
      RecordAccumulator.java  763  abortBatches(...)
      RecordAccumulator.java:763:in `abortBatches'
      More (5 lines)
      Nested Exceptionsorg.apache.kafka.streams.errors.StreamsException: Error encountered sending record to topic ola-update-1 for task 4_7 due to: org.apache.kafka.common.errors.InvalidPidMappingException: The producer attempted to use a producer id which is not currently assigned to its transactional id. Exception handler choose to FAIL the processing, no more records would be sent.
      RecordCollectorImpl.java  226  recordSendError(...)
      RecordCollectorImpl.java:226:in `recordSendError'
       org.apache.kafka.common.errors.InvalidPidMappingException: The producer attempted to use a producer id which is not currently assigned to its transactional id.
      

      I've seen that KAFKA-6821 described the same problem on an earlier version of Kafka and was closed due to the subsequent works on EOS.

      Another ticket raised recently shows that the exception is still occurring (but the ticket wasn't raised for that specific error): KAFKA-12774

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              gphilipp Gilles Philippart
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated: