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

NullPointerException when loading producer snapshot

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Blocker
    • Resolution: Duplicate
    • None
    • None
    • None

    Description

      This was seen in the server.log of a recent system test failure:

      java.lang.NullPointerException
              at kafka.log.Log.updateFirstUnstableOffset(Log.scala:689)
              at kafka.log.Log.loadProducerState(Log.scala:466)
              at kafka.log.Log.<init>(Log.scala:185)
              at kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$5$$anonfun$apply$12$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:172)
              at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:57)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
              at java.util.concurrent.FutureTask.run(FutureTask.java:262)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              at java.lang.Thread.run(Thread.java:745)
      

      The actual snapshot file had these contents:

      amehta-macbook-pro:kafka-data-logs apurva$ ~/workspace/confluent/kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments -files output-topic-1/00000000000000005776.snapshot
      Dumping output-topic-1/00000000000000005776.snapshot
      producerId: 1001 producerEpoch: 0 lastSequence: 3249 lastOffset: 6026 offsetDelta: 0 lastTimestamp: 1497127651606 coordinatorEpoch: 2 currentTxnFirstOffset: None
      producerId: 0 producerEpoch: 0 lastSequence: 2749 lastOffset: 6025 offsetDelta: 249 lastTimestamp: 1497127647269 coordinatorEpoch: 1 currentTxnFirstOffset: Some(5776)
      

      The log file ended at offset 5775:

      baseOffset: 4770 lastOffset: 4770 baseSequence: -1 lastSequence: -1 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 58134 CreateTime: 1497127646732 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 3476040058
      baseOffset: 4771 lastOffset: 5020 baseSequence: 2750 lastSequence: 2999 producerId: 1001 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 58212 CreateTime: 1497127646719 isvalid: true size: 2997 magic: 2 compresscodec: NONE crc: 1010131725
      baseOffset: 5021 lastOffset: 5021 baseSequence: -1 lastSequence: -1 producerId: 1001 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 61209 CreateTime: 1497127646923 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 4001115747
      baseOffset: 5022 lastOffset: 5271 baseSequence: 2000 lastSequence: 2249 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 61287 CreateTime: 1497127646760 isvalid: true size: 2997 magic: 2 compresscodec: NONE crc: 2903697477
      baseOffset: 5272 lastOffset: 5272 baseSequence: -1 lastSequence: -1 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 64284 CreateTime: 1497127646982 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 2599716061
      baseOffset: 5273 lastOffset: 5522 baseSequence: 3000 lastSequence: 3249 producerId: 1001 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 64362 CreateTime: 1497127646948 isvalid: true size: 3080 magic: 2 compresscodec: NONE crc: 3176858298
      baseOffset: 5523 lastOffset: 5523 baseSequence: -1 lastSequence: -1 producerId: 1001 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 67442 CreateTime: 1497127647152 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 3464913172
      baseOffset: 5524 lastOffset: 5773 baseSequence: 2250 lastSequence: 2499 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 67520 CreateTime: 1497127647001 isvalid: true size: 3081 magic: 2 compresscodec: NONE crc: 712707513
      baseOffset: 5774 lastOffset: 5774 baseSequence: -1 lastSequence: -1 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 70601 CreateTime: 1497127647248 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 1043637525
      baseOffset: 5775 lastOffset: 5775 baseSequence: -1 lastSequence: -1 producerId: 1001 producerEpoch: 0 partitionLeaderEpoch: 1 isTransactional: true position: 70679 CreateTime: 1497127647352 isvalid: true size: 78 magic: 2 compresscodec: NONE crc: 968257373
      

      The suspicious bit is the `currentTxnFirstOffset: Some(5776)`. This indicates that snapshot didn't get truncated correctly since it retains an offset which doesn't exist in the truncated segment, causing an NPE.

      Attachments

        Issue Links

          Activity

            People

              hachikuji Jason Gustafson
              apurva Apurva Mehta
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: