@@ -27,7 +27,6 @@ import kafka.server.metadata.KRaftMetadataCache
2727import kafka .server .share .DelayedShareFetch
2828import kafka .utils .CoreUtils .{inReadLock , inWriteLock }
2929import kafka .utils ._
30- import kafka .zookeeper .ZooKeeperClientException
3130import org .apache .kafka .common .{DirectoryId , IsolationLevel , TopicIdPartition , TopicPartition , Uuid }
3231import org .apache .kafka .common .errors ._
3332import org .apache .kafka .common .message .AlterPartitionRequestData .BrokerState
@@ -767,14 +766,7 @@ class Partition(val topicPartition: TopicPartition,
767766 LeaderRecoveryState .RECOVERED
768767 )
769768
770- try {
771- createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetDirectoryId)
772- } catch {
773- case e : ZooKeeperClientException =>
774- stateChangeLogger.error(s " A ZooKeeper client exception has occurred and makeLeader will be skipping the " +
775- s " state change for the partition $topicPartition with leader epoch: $leaderEpoch. " , e)
776- return false
777- }
769+ createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetDirectoryId)
778770
779771 val leaderLog = localLogOrException
780772
@@ -868,14 +860,7 @@ class Partition(val topicPartition: TopicPartition,
868860 LeaderRecoveryState .of(partitionState.leaderRecoveryState)
869861 )
870862
871- try {
872- createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetLogDirectoryId)
873- } catch {
874- case e : ZooKeeperClientException =>
875- stateChangeLogger.error(s " A ZooKeeper client exception has occurred. makeFollower will be skipping the " +
876- s " state change for the partition $topicPartition with leader epoch: $leaderEpoch. " , e)
877- return false
878- }
863+ createLogInAssignedDirectoryId(partitionState, highWatermarkCheckpoints, topicId, targetLogDirectoryId)
879864
880865 val followerLog = localLogOrException
881866 if (isNewLeaderEpoch) {
0 commit comments