soarez commented on code in PR #15951: URL: https://github.com/apache/kafka/pull/15951#discussion_r1600179621
########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -2114,19 +2114,16 @@ class ReplicaManager(val config: KafkaConfig, partition.log.foreach { _ => val leader = BrokerEndPoint(config.brokerId, "localhost", -1) - // Add future replica log to partition's map - partition.createLogIfNotExists( - isNew = false, - isFutureReplica = true, - offsetCheckpoints, - topicIds(partition.topic)) - - // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move - // replica from source dir to destination dir - logManager.abortAndPauseCleaning(topicPartition) - - futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(topicIds(topicPartition.topic), leader, - partition.getLeaderEpoch, futureLog.highWatermark)) + if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { Review Comment: Sincre this section is inside a block of `logManager.getLog(topicPartition, isFuture = true).foreach { futureLog =>`, doesn't that mean this only runs if the future replica exists? ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -2114,19 +2114,16 @@ class ReplicaManager(val config: KafkaConfig, partition.log.foreach { _ => val leader = BrokerEndPoint(config.brokerId, "localhost", -1) - // Add future replica log to partition's map - partition.createLogIfNotExists( - isNew = false, - isFutureReplica = true, - offsetCheckpoints, - topicIds(partition.topic)) - - // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move - // replica from source dir to destination dir - logManager.abortAndPauseCleaning(topicPartition) - - futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(topicIds(topicPartition.topic), leader, - partition.getLeaderEpoch, futureLog.highWatermark)) + if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { + // Add future replica log to partition's map Review Comment: I assume this comment refers to setting (populating) `partition.futureLog`, which is only written to via a call `partition.createLogIfNotExists()`. Since we're replacing the call to `partition.createLogIfNotExists()` with `partition.maybeCreateFutureReplica()`, this comment should be moved there? ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -2114,19 +2114,16 @@ class ReplicaManager(val config: KafkaConfig, partition.log.foreach { _ => val leader = BrokerEndPoint(config.brokerId, "localhost", -1) - // Add future replica log to partition's map - partition.createLogIfNotExists( - isNew = false, - isFutureReplica = true, - offsetCheckpoints, - topicIds(partition.topic)) - - // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move - // replica from source dir to destination dir - logManager.abortAndPauseCleaning(topicPartition) - - futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(topicIds(topicPartition.topic), leader, - partition.getLeaderEpoch, futureLog.highWatermark)) + if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { Review Comment: `partition.maybeCreateFutureReplica()` only calls `partition.createLogIfNotExists()` if the future replica doesn't yet exist. If that happens, `partition.futureLog` won't be set, so we need to call `partition.setLog(futureLog, true)`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org