You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/12/14 03:26:42 UTC

[GitHub] [pulsar] heesung-sn commented on a diff in pull request #18858: [improve][broker] Write the child ownership to `ServiceUnitStateChannel` instead of ZK when handling bundle split

heesung-sn commented on code in PR #18858:
URL: https://github.com/apache/pulsar/pull/18858#discussion_r1047958331


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {

Review Comment:
   In this case, do we need to change the bundle state back to `Splitting -> Owned`?



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java:
##########
@@ -566,7 +602,7 @@ public void handleBrokerDeletionEventTest()
         assertEquals(0, followerCleanupJobs.size());
         assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCnt"));
         assertEquals(1, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt"));
-        assertEquals(2, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt"));
+        assertEquals(4, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt"));

Review Comment:
   why do we see +2 here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());

Review Comment:
   splitBundles.left and right are not very clear. Can we assign vars with meaningful names for left and right for better readability? 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());
+            for (NamespaceBundle sBundle : splitBundles.getRight()) {
+                futures.add(pubAsync(sBundle.toString(), next).thenAccept(__ -> {}));
+            }
+            NamespaceName nsname = bundle.getNamespaceObject();
+            FutureUtil.waitForAll(futures).thenRun(() ->

Review Comment:
   Should we wait til getOwner(newBundles) == this.broker?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());
+            for (NamespaceBundle sBundle : splitBundles.getRight()) {
+                futures.add(pubAsync(sBundle.toString(), next).thenAccept(__ -> {}));
+            }
+            NamespaceName nsname = bundle.getNamespaceObject();
+            FutureUtil.waitForAll(futures).thenRun(() ->
+                    namespaceService.updateNamespaceBundles(nsname, splitBundles.getLeft()).thenCompose(__ ->
+                            namespaceService.updateNamespaceBundlesForPolicies(nsname, splitBundles.getLeft()))
+                    .thenRun(() -> {
+                        bundleFactory.invalidateBundleCache(bundle.getNamespaceObject());
+                        updateFuture.complete(splitBundles.getRight());
+                    }).exceptionally(ex -> {
+                        log.warn("Failed to update namespace policies [{}], NamespaceBundle: {} due to {}",
+                                nsname, bundle.getBundleRange(), ex.getMessage());
+                        updateFuture.completeExceptionally(ex);
+                        return null;
+                    })).exceptionally(e -> {
+                        updateFuture.completeExceptionally(e);
+                        return null;
+                    });
+        }).exceptionally(e -> {
+            updateFuture.completeExceptionally(e);
+            return null;
+        });
+
+        updateFuture.thenAccept(r -> {
+            // Free the old bundle
+            tombstoneAsync(serviceUnit).thenRun(() -> {
+                // Update bundled_topic cache for load-report-generation
+                pulsar.getBrokerService().refreshTopicToStatsMaps(bundle);
+                // TODO: Update the load data immediately if needed.
+                completionFuture.complete(null);
+                double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime));
+                log.info("Successfully split {} parent namespace-bundle to {} in {} ms", serviceUnit, r,
+                        splitBundleTime);
+            }).exceptionally(e -> {
+                double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime));
+                String msg = format("Failed to free bundle %s in %s ms, under namespace [%s] with error %s",
+                        bundle.getNamespaceObject().toString(), splitBundleTime, bundle, e.getMessage());
+                completionFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return null;
+            });
+        }).exceptionally(ex -> {
+            // Retry several times on BadVersion
+            if ((ex.getCause() instanceof MetadataStoreException.BadVersionException)

Review Comment:
   Where do we get this `MetadataStoreException.BadVersionException` from?  it seems like we are not accessing the metadata store in this block.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());
+            for (NamespaceBundle sBundle : splitBundles.getRight()) {
+                futures.add(pubAsync(sBundle.toString(), next).thenAccept(__ -> {}));
+            }
+            NamespaceName nsname = bundle.getNamespaceObject();
+            FutureUtil.waitForAll(futures).thenRun(() ->
+                    namespaceService.updateNamespaceBundles(nsname, splitBundles.getLeft()).thenCompose(__ ->
+                            namespaceService.updateNamespaceBundlesForPolicies(nsname, splitBundles.getLeft()))
+                    .thenRun(() -> {
+                        bundleFactory.invalidateBundleCache(bundle.getNamespaceObject());
+                        updateFuture.complete(splitBundles.getRight());
+                    }).exceptionally(ex -> {
+                        log.warn("Failed to update namespace policies [{}], NamespaceBundle: {} due to {}",
+                                nsname, bundle.getBundleRange(), ex.getMessage());
+                        updateFuture.completeExceptionally(ex);
+                        return null;
+                    })).exceptionally(e -> {
+                        updateFuture.completeExceptionally(e);
+                        return null;
+                    });
+        }).exceptionally(e -> {
+            updateFuture.completeExceptionally(e);
+            return null;
+        });
+
+        updateFuture.thenAccept(r -> {
+            // Free the old bundle
+            tombstoneAsync(serviceUnit).thenRun(() -> {
+                // Update bundled_topic cache for load-report-generation
+                pulsar.getBrokerService().refreshTopicToStatsMaps(bundle);
+                // TODO: Update the load data immediately if needed.
+                completionFuture.complete(null);
+                double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime));
+                log.info("Successfully split {} parent namespace-bundle to {} in {} ms", serviceUnit, r,
+                        splitBundleTime);
+            }).exceptionally(e -> {
+                double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime));
+                String msg = format("Failed to free bundle %s in %s ms, under namespace [%s] with error %s",
+                        bundle.getNamespaceObject().toString(), splitBundleTime, bundle, e.getMessage());
+                completionFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return null;
+            });
+        }).exceptionally(ex -> {
+            // Retry several times on BadVersion
+            if ((ex.getCause() instanceof MetadataStoreException.BadVersionException)
+                    && (counter.incrementAndGet() < NamespaceService.BUNDLE_SPLIT_RETRY_LIMIT)) {
+                pulsar.getExecutor().schedule(() -> pulsar.getOrderedExecutor()
+                                .execute(() -> splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle,
+                                        serviceUnit, data, counter, startTime, completionFuture)),
+                        100, MILLISECONDS);
+            } else if (ex instanceof IllegalArgumentException) {
+                completionFuture.completeExceptionally(ex);
+            } else {
+                // Retry enough, or meet other exception
+                String msg = format("Bundle: %s not success update nsBundles, counter %d, reason %s",
+                        bundle.toString(), counter.get(), ex.getMessage());
+                completionFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+            }
+            return null;
+        });
+    }
+    private CompletableFuture<Pair<NamespaceBundles, List<NamespaceBundle>>> getSplitBoundary(NamespaceBundle bundle) {

Review Comment:
   Can we define this `getSplitBoundary` in NamespaceBundleFactory?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());
+            for (NamespaceBundle sBundle : splitBundles.getRight()) {
+                futures.add(pubAsync(sBundle.toString(), next).thenAccept(__ -> {}));
+            }
+            NamespaceName nsname = bundle.getNamespaceObject();
+            FutureUtil.waitForAll(futures).thenRun(() ->
+                    namespaceService.updateNamespaceBundles(nsname, splitBundles.getLeft()).thenCompose(__ ->

Review Comment:
    Can't we flatten these nested futures?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -510,25 +524,120 @@ private CompletableFuture<Integer> closeServiceUnit(String serviceUnit) {
                 });
     }
 
-    private CompletableFuture<Void> splitServiceUnit(String serviceUnit) {
-        // TODO: after the split we need to write the child ownerships to BSC instead of ZK.
+    private CompletableFuture<Void> splitServiceUnit(String serviceUnit, ServiceUnitStateData data) {
+        // Write the child ownerships to BSC.
         long startTime = System.nanoTime();
-        return pulsar.getNamespaceService()
-                .splitAndOwnBundle(getNamespaceBundle(serviceUnit),
-                        false,
-                        NamespaceBundleSplitAlgorithm.of(pulsar.getConfig().getDefaultNamespaceBundleSplitAlgorithm()),
-                        null)
-                .whenComplete((__, ex) -> {
-                    double splitBundleTime = TimeUnit.NANOSECONDS
-                            .toMillis((System.nanoTime() - startTime));
-                    if (ex == null) {
-                        log.info("Successfully split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime);
-                    } else {
-                        log.error("Failed to split {} namespace-bundle in {} ms",
-                                serviceUnit, splitBundleTime, ex);
-                    }
-                });
+        NamespaceService namespaceService = pulsar.getNamespaceService();
+        NamespaceBundleFactory bundleFactory = namespaceService.getNamespaceBundleFactory();
+        NamespaceBundle bundle = getNamespaceBundle(serviceUnit);
+        CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+        final AtomicInteger counter = new AtomicInteger(0);
+        this.splitServiceUnitOnceAndRetry(namespaceService, bundleFactory, bundle, serviceUnit, data,
+                counter, startTime, completionFuture);
+        return completionFuture;
+    }
+
+    @VisibleForTesting
+    protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
+                                                NamespaceBundleFactory bundleFactory,
+                                                NamespaceBundle bundle,
+                                                String serviceUnit,
+                                                ServiceUnitStateData data,
+                                                AtomicInteger counter,
+                                                long startTime,
+                                                CompletableFuture<Void> completionFuture) {
+        CompletableFuture<List<NamespaceBundle>> updateFuture = new CompletableFuture<>();
+
+        getSplitBoundary(bundle).thenAccept(splitBundles -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundles == null) {
+                String msg = format("Bundle %s not found under namespace", serviceUnit);
+                updateFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException(msg));
+                return;
+            }
+            List<CompletableFuture<Void>> futures = new ArrayList<>();
+            ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker());
+            for (NamespaceBundle sBundle : splitBundles.getRight()) {
+                futures.add(pubAsync(sBundle.toString(), next).thenAccept(__ -> {}));
+            }
+            NamespaceName nsname = bundle.getNamespaceObject();
+            FutureUtil.waitForAll(futures).thenRun(() ->
+                    namespaceService.updateNamespaceBundles(nsname, splitBundles.getLeft()).thenCompose(__ ->
+                            namespaceService.updateNamespaceBundlesForPolicies(nsname, splitBundles.getLeft()))
+                    .thenRun(() -> {
+                        bundleFactory.invalidateBundleCache(bundle.getNamespaceObject());
+                        updateFuture.complete(splitBundles.getRight());
+                    }).exceptionally(ex -> {
+                        log.warn("Failed to update namespace policies [{}], NamespaceBundle: {} due to {}",
+                                nsname, bundle.getBundleRange(), ex.getMessage());
+                        updateFuture.completeExceptionally(ex);
+                        return null;
+                    })).exceptionally(e -> {
+                        updateFuture.completeExceptionally(e);

Review Comment:
   After flattening the futures, I think we can simplify these exceptionally blocks.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org