chia7712 commented on code in PR #21363:
URL: https://github.com/apache/kafka/pull/21363#discussion_r2733039289


##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -1072,33 +1070,22 @@ public void onResignation(int partitionIndex, 
OptionalInt partitionLeaderEpoch)
         );
     }
 
-    @Override
-    public void onTopicsDeleted(Set<Uuid> deletedTopicIds, BufferSupplier 
bufferSupplier) throws ExecutionException, InterruptedException {
-        throwIfNotActive();
-        if (deletedTopicIds.isEmpty()) {
-            return;
-        }
-        CompletableFuture.allOf(
-            FutureUtils.mapExceptionally(
-                runtime.scheduleWriteAllOperation(
-                    "on-topics-deleted",
-                    coordinator -> 
coordinator.maybeCleanupShareState(deletedTopicIds)
-                ),
-                exception -> {
-                    log.error("Received error while trying to cleanup deleted 
topics.", exception);
-                    return null;
-                }
-            ).toArray(new CompletableFuture<?>[0])
-        ).get();
-    }
-
     @Override
     public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage) {
         throwIfNotActive();
+        Objects.requireNonNull(delta, "delta must be provided");
+        Objects.requireNonNull(newImage, "newImage must be provided");
+
         this.runtime.onMetadataUpdate(
-            new KRaftCoordinatorMetadataDelta(Objects.requireNonNull(delta, 
"delta must be provided")),
-            new KRaftCoordinatorMetadataImage(Objects.requireNonNull(newImage, 
"newImage must be provided"))
+            new KRaftCoordinatorMetadataDelta(delta),
+            new KRaftCoordinatorMetadataImage(newImage)
         );
+
+        // Handle topic deletions from the delta.
+        if (delta.topicsDelta() != null && 
!delta.topicsDelta().deletedTopicIds().isEmpty()) {
+            handleTopicsDeletion(delta.topicsDelta().deletedTopicIds());

Review Comment:
   Previously, `handleTopicsDeletion` executed before `onMetadataUpdate`, but 
now the order is reversed. I think the current order makes more sense



-- 
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