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

TestRaftServer's scala.MatchError: null on test-kraft-server-start.sh

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.0.0
    • kraft
    • None

    Description

      Encounter the following exception when trying to run the TestRaftServer:

      bin/test-kraft-server-start.sh --config config/kraft.properties
      [2021-06-14 17:15:43,232] ERROR [raft-workload-generator]: Error due to (kafka.tools.TestRaftServer$RaftWorkloadGenerator)
       scala.MatchError: null
       at kafka.tools.TestRaftServer$RaftWorkloadGenerator.doWork(TestRaftServer.scala:220)
       at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96)
       [2021-06-14 17:15:43,253] INFO [raft-workload-generator]: Stopped (kafka.tools.TestRaftServer$RaftWorkloadGenerator)

      That happens on the followin match:

      eventQueue.poll(eventTimeoutMs, TimeUnit.MILLISECONDS) match {
        case HandleClaim(epoch) =>
            claimedEpoch = Some(epoch)
            throttler.reset()
            pendingAppends.clear()
            recordCount.set(0)    
        case HandleResign =>
            claimedEpoch = None
            pendingAppends.clear()    case HandleCommit(reader) =>
            try {
              while (reader.hasNext) {
                val batch = reader.next()
                claimedEpoch.foreach { leaderEpoch =>
                  handleLeaderCommit(leaderEpoch, batch)
                }
              }
            } finally {
              reader.close()
            }    
        case HandleSnapshot(reader) =>
            // Ignore snapshots; only interested in records appended by this leader
            reader.close()    
        case Shutdown => // Ignore shutdown command
      }
      

      Full log attached. When the eventQueue.poll returns null (if deque is empty), there isn't a case to match so the thread gets stuck and stops processing events (raft-workload-generator).

      Proposal:
      Add a case null to the match so the raft-workload-generator can continue.

      Attachments

        1. TestRaftServer.log
          23 kB
          Ignacio Acuna

        Issue Links

          Activity

            People

              IgnacioAcunaFri Ignacio Acuna
              IgnacioAcunaFri Ignacio Acuna
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: