Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-18532: Clean Partition.scala zookeeper logic #18594

Merged
merged 1 commit into from
Jan 19, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 2 additions & 17 deletions core/src/main/scala/kafka/cluster/Partition.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import kafka.server.metadata.KRaftMetadataCache
import kafka.server.share.DelayedShareFetch
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils._
import kafka.zookeeper.ZooKeeperClientException
import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.errors._
import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState
Expand Down Expand Up @@ -767,14 +766,7 @@ class Partition(val topicPartition: TopicPartition,
LeaderRecoveryState.RECOVERED
)

try {
createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetDirectoryId)
} 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
}
createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetDirectoryId)

val leaderLog = localLogOrException

Expand Down Expand Up @@ -868,14 +860,7 @@ class Partition(val topicPartition: TopicPartition,
LeaderRecoveryState.of(partitionState.leaderRecoveryState)
)

try {
createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetLogDirectoryId)
} 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
}
createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetLogDirectoryId)

val followerLog = localLogOrException
if (isNewLeaderEpoch) {
Expand Down
3 changes: 1 addition & 2 deletions core/src/main/scala/kafka/server/ReplicaManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2833,7 +2833,7 @@ class ReplicaManager(val config: KafkaConfig,
val leaderChangedPartitions = new mutable.HashSet[Partition]
val followerChangedPartitions = new mutable.HashSet[Partition]
if (!localChanges.leaders.isEmpty) {
applyLocalLeadersDelta(leaderChangedPartitions, newImage, delta, lazyOffsetCheckpoints, localChanges.leaders.asScala, localChanges.directoryIds.asScala)
applyLocalLeadersDelta(leaderChangedPartitions, delta, lazyOffsetCheckpoints, localChanges.leaders.asScala, localChanges.directoryIds.asScala)
}
if (!localChanges.followers.isEmpty) {
applyLocalFollowersDelta(followerChangedPartitions, newImage, delta, lazyOffsetCheckpoints, localChanges.followers.asScala, localChanges.directoryIds.asScala)
Expand All @@ -2857,7 +2857,6 @@ class ReplicaManager(val config: KafkaConfig,

private def applyLocalLeadersDelta(
changedPartitions: mutable.Set[Partition],
newImage: MetadataImage,
delta: TopicsDelta,
offsetCheckpoints: OffsetCheckpoints,
localLeaders: mutable.Map[TopicPartition, LocalReplicaChanges.PartitionInfo],
Expand Down
Loading