You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Ignacio Acuna (Jira)" <ji...@apache.org> on 2021/06/15 00:20:00 UTC

[jira] [Updated] (KAFKA-12949) TestRaftServer's scala.MatchError: null on test-kraft-server-start.sh

     [ https://issues.apache.org/jira/browse/KAFKA-12949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ignacio Acuna updated KAFKA-12949:
----------------------------------
    Attachment: TestRaftServer.log

> TestRaftServer's scala.MatchError: null on test-kraft-server-start.sh
> ---------------------------------------------------------------------
>
>                 Key: KAFKA-12949
>                 URL: https://issues.apache.org/jira/browse/KAFKA-12949
>             Project: Kafka
>          Issue Type: Bug
>          Components: kraft
>            Reporter: Ignacio Acuna
>            Assignee: Ignacio Acuna
>            Priority: Major
>         Attachments: TestRaftServer.log
>
>
> Encounter the following exception when trying to run the TestRaftServer:
> {code:java}
> bin/test-kraft-server-start.sh --config config/kraft.properties{code}
> {code:java}
> [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){code}
> That happens on the followin match:
> {code:java}
> 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
> }
> {code}
> 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.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)