You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by "Demogorgon314 (via GitHub)" <gi...@apache.org> on 2023/02/06 12:28:15 UTC

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

Demogorgon314 commented on code in PR #18858:
URL: https://github.com/apache/pulsar/pull/18858#discussion_r1097315839


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -512,25 +521,107 @@ 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<>();
+
+        pulsar.getNamespaceService().getSplitBoundary(bundle, null).thenAccept(splitBundlesPair -> {
+            // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper.
+            if (splitBundlesPair == 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:
   Yes, we can also specify the new bundles and the destination broker `org.apache.pulsar.broker.loadbalance.extensions.models.Split#splitServiceUnitToDestBroker`. This feature will be implemented in another PR.



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