Browse Source

KAFKA-9769: ReplicaManager Partition.makeFollower Increases LeaderEpoch when ZooKeeper disconnect occurs (#8479)

Skip the partition incurring ZooKeeper exception when becoming leader or follower.

Reviewers: Joel Koshy <jjkoshy@gmail.com>,  Jun Rao <junrao@gmail.com>
pull/8974/head
Andrew Choi 4 years ago committed by GitHub
parent
commit
3a55fb9f28
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 21
      core/src/main/scala/kafka/cluster/Partition.scala

21
core/src/main/scala/kafka/cluster/Partition.scala

@ -29,6 +29,7 @@ import kafka.server.checkpoints.OffsetCheckpoints @@ -29,6 +29,7 @@ import kafka.server.checkpoints.OffsetCheckpoints
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils._
import kafka.zk.{AdminZkClient, KafkaZkClient}
import kafka.zookeeper.ZooKeeperClientException
import org.apache.kafka.common.errors._
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.protocol.Errors
@ -499,7 +500,15 @@ class Partition(val topicPartition: TopicPartition, @@ -499,7 +500,15 @@ class Partition(val topicPartition: TopicPartition,
addingReplicas = addingReplicas,
removingReplicas = removingReplicas
)
createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
try {
createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
} catch {
case e: ZooKeeperClientException =>
stateChangeLogger.error(s"A ZooKeeper client exception has occurred and makeLeader will be skipping the " +
s"state change for the partition $topicPartition with leader epoch: $leaderEpoch ", e)
return false
}
val leaderLog = localLogOrException
val leaderEpochStartOffset = leaderLog.logEndOffset
@ -570,7 +579,15 @@ class Partition(val topicPartition: TopicPartition, @@ -570,7 +579,15 @@ class Partition(val topicPartition: TopicPartition,
addingReplicas = partitionState.addingReplicas.asScala.map(_.toInt),
removingReplicas = partitionState.removingReplicas.asScala.map(_.toInt)
)
createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
try {
createLogIfNotExists(partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints)
} catch {
case e: ZooKeeperClientException =>
stateChangeLogger.error(s"A ZooKeeper client exception has occurred. makeFollower will be skipping the " +
s"state change for the partition $topicPartition with leader epoch: $leaderEpoch.", e)
return false
}
val followerLog = localLogOrException
val leaderEpochEndOffset = followerLog.logEndOffset

Loading…
Cancel
Save