soarez commented on code in PR #14881:
URL: https://github.com/apache/kafka/pull/14881#discussion_r1420734792


##########
metadata/src/main/java/org/apache/kafka/image/TopicDelta.java:
##########
@@ -158,11 +160,21 @@ public LocalReplicaChanges localChanges(int brokerId) {
                         new LocalReplicaChanges.PartitionInfo(id(), 
entry.getValue())
                     );
                     topicIds.putIfAbsent(name(), id());
+                    updateDirectoryIds(brokerId, entry, prevPartition, 
directoryIds);
                 }
             }
         }
 
-        return new LocalReplicaChanges(deletes, leaders, followers, topicIds);
+        return new LocalReplicaChanges(deletes, leaders, followers, topicIds, 
directoryIds);
+    }
+
+    private void updateDirectoryIds(int brokerId, Entry<Integer, 
PartitionRegistration> entry, PartitionRegistration prevPartition, 
Map<TopicPartition, LocalReplicaChanges.PartitionInfo> directoryIds) {
+        if (prevPartition == null || prevPartition.directory(brokerId) != 
entry.getValue().directory(brokerId)) {
+            directoryIds.put(
+                    new TopicPartition(name(), entry.getKey()),
+                    new LocalReplicaChanges.PartitionInfo(id(), 
entry.getValue())

Review Comment:
   I think this can just be `entry.getValue().directory(brokerId)`, we don't 
need the rest



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -2626,8 +2601,16 @@ class ReplicaManager(val config: KafkaConfig,
     localLeaders.forKeyValue { (tp, info) =>
       getOrCreatePartition(tp, delta, info.topicId).foreach { case (partition, 
isNew) =>
         try {
+          val partitionDirectoryId: Option[Uuid] =
+            Option(newImage.topics().getPartition(info.topicId(), 
partition.partitionId)).map(_.directory(localBrokerId))

Review Comment:
   Maybe we can default this to MIGRATING instead of an Option?
   
   



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -2626,8 +2601,16 @@ class ReplicaManager(val config: KafkaConfig,
     localLeaders.forKeyValue { (tp, info) =>
       getOrCreatePartition(tp, delta, info.topicId).foreach { case (partition, 
isNew) =>
         try {
+          val partitionDirectoryId: Option[Uuid] =
+            Option(newImage.topics().getPartition(info.topicId(), 
partition.partitionId)).map(_.directory(localBrokerId))
+
           val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew)
-          partition.makeLeader(state, offsetCheckpoints, Some(info.topicId))
+          partition.makeLeader(state, offsetCheckpoints, Some(info.topicId), 
partitionDirectoryId)
+
+          if (!delta.localChanges(localBrokerId).directoryIds().isEmpty) {
+            maybeUpdateTopicAssignment(partition, partitionDirectoryId)
+          }

Review Comment:
   This can be pulled up to `def applyDelta(delta: TopicsDelta, newImage: 
MetadataImage):`.
   Maybe we can do 
`localChanges.directoryIds().forEach(maybeUpdateTopicAssignment)` after 
   ```
           if (!localChanges.leaders.isEmpty) {
             applyLocalLeadersDelta(leaderChangedPartitions, newImage, delta, 
lazyOffsetCheckpoints, localChanges.leaders.asScala)
           }
           if (!localChanges.followers.isEmpty) {
             applyLocalFollowersDelta(followerChangedPartitions, newImage, 
delta, lazyOffsetCheckpoints, localChanges.followers.asScala)
           }
           localChanges.directoryIds().forEach(maybeUpdateTopicAssignment)
   ```
   
   WDYT?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to