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/29 07:56:19 UTC

[GitHub] [pulsar] Demogorgon314 opened a new pull request, #19102: [improve][broker] PIP-192: Implement extensible load manager

Demogorgon314 opened a new pull request, #19102:
URL: https://github.com/apache/pulsar/pull/19102

   PIP: #16691
   
   ### Motivation
   
   Implement extensible load manager.
   
   ### Modifications
   
   For the PIP-192, this PR adds `ExtensibleLoadManagerImpl` and unit tests.
   
   This PR also changes:
   1. Added `CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
               Optional<ServiceUnitId> topic, ServiceUnitId bundle)` to `LoadManager ` interface.
   2. Added `CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle)` to `LoadManager` interface.
   3. Change `CompletableFuture<String> getOwnerAsync(String serviceUnit)` to `CompletableFuture<Optional<String>> getOwnerAsync(String serviceUnit)` to unify the result.
   
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [ ] `doc` <!-- Your PR contains doc changes. Please attach the local preview screenshots (run `sh start.sh` at `pulsar/site2/website`) to your PR description, or else your PR might not get merged. -->
   - [ ] `doc-required` <!-- Your PR changes impact docs and you will update later -->
   - [x] `doc-not-needed` <!-- Your PR changes do not impact docs -->
   - [ ] `doc-complete` <!-- Docs have been already added -->
   
   ### Matching PR in forked repository
   
   PR in forked repository: https://github.com/Demogorgon314/pulsar/pull/10 <!-- ENTER URL HERE -->
   
   <!--
   After opening this PR, the build in apache/pulsar will fail and instructions will
   be provided for opening a PR in the PR author's forked repository.
   
   apache/pulsar pull requests should be first tested in your own fork since the 
   apache/pulsar CI based on GitHub Actions has constrained resources and quota.
   GitHub Actions provides separate quota for pull requests that are executed in 
   a forked repository.
   
   The tests will be run in the forked repository until all PR review comments have
   been handled, the tests pass and the PR is approved by a reviewer.
   -->
   


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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061331104


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }

Review Comment:
   Throwing an unchecked exception in the callback of `CompletableFuture` is not a good manner. The caller usually prints the stack of the exception (in `exceptionally` or `whenComplete`). But the stack is meaningless. A better practice is to print an error log here and return an empty `Optional`.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1066087400


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   nit: It would be great if we can have a separate PR to integrate ExtensibleLoadManagerImpl into the existing code(to the NamespaceService and PulsarService). This PR could be ExtensibleLoadManagerImpl (and other dependent changes) only.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1093985988


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();

Review Comment:
   I have rechecked the `start`, `initialize`, and `close` methods, the `start` and `initialize` methods always call in the same thread, so we don't need to add the synchronized modifier.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061286163


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private volatile boolean started = false;

Review Comment:
   ```suggestion
       private boolean started = false;
   ```
   
   The `start` and `close` methods are both synchronized, so we could remove the `volatile`.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private volatile boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {

Review Comment:
   ```suggestion
       public synchronized void initialize(PulsarService pulsar) {
   ```
   
   If you expect `start` to be called in different threads, the `initialize` method should be synchronized as well.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;

Review Comment:
   ```suggestion
       private final List<BrokerFilter> brokerFilterPipeline;
   ```
   
   Maybe we can initialize this field here instead of the constructor



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1060229456


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   Good point. I'll add it later.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059232448


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -242,11 +242,12 @@ public CompletableFuture<Optional<String>> getChannelOwnerAsync() {
                     // TODO: discard this protocol prefix removal
                     //  by a util func that returns lookupServiceAddress(serviceUrl)
                     if (leader.isPresent()) {
-                        String broker = leader.get().getServiceUrl();
-                        broker = broker.substring(broker.lastIndexOf('/') + 1);
-                        return Optional.of(broker);
+                        return Optional.of(leader.get().getServiceUrl());

Review Comment:
   Let's change it back, I find the `ServiceUnitStateChannelTest` will fail after this change because the `PulsarService` started a `LeaderElectionService`



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1094032298


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    public static final String BROKER_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-broker-load-data").toString();
+
+    public static final String TOP_BUNDLES_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-top-bundles-load-data").toString();
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    private LoadDataStore<BrokerLoadData> brokerLoadDataStore;
+    private LoadDataStore<TopBundlesLoadData> topBundlesLoadDataStore;

Review Comment:
   Sorry, I forgot to close the load data store. Now it is used in `start` and `close` method.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059106101


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -794,6 +802,12 @@ public void start() throws PulsarServerException {
             }
             brokerService.start();
 
+            if (this.loadManager.get() instanceof ExtensibleLoadManagerWrapper) {

Review Comment:
   Let's define a static function and globally use it from PulsarService, NamespaceService, and other places.
   
   Proposal:
   ```
   ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(ServiceConfiguration conf){
      return ExtensibleLoadManagerImpl.class.getname().equals(conf.getLoadManagerClassName())
   }
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1085,6 +1129,9 @@ protected void closeLocalMetadataStore() throws Exception {
     }
 
     protected void startLeaderElectionService() {
+        if (this.loadManager.get() instanceof ExtensibleLoadManagerWrapper) {
+            return;

Review Comment:
   Use the global `isLoadManagerExtensionEnabled ` static func, as discussed above.
   
   Also, plz add a log.
   log.info("The load manager extension is enabled. Skipping PulsarService LeaderElectionService.")



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -825,6 +839,11 @@ public void start() throws PulsarServerException {
                 this.webSocketService.setLocalCluster(clusterData);
             }
 
+            // By starting the Load manager service, the broker will also become visible
+            // to the rest of the broker by creating the registration z-node. This needs
+            // to be done only when the broker is fully operative.
+            this.startLoadManagementService();

Review Comment:
   Please add a comment that the load manager service and its service unit state channel need to be initialized first( namespace service depends on load manager)



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -925,6 +939,36 @@ public void start() throws PulsarServerException {
         }
     }
 
+    protected void createNamespaceIfNotExists(String cluster, String publicTenant, NamespaceName ns) throws Exception {
+        ClusterResources cr = this.getPulsarResources().getClusterResources();
+        TenantResources tr = this.getPulsarResources().getTenantResources();
+        NamespaceResources nsr = this.getPulsarResources().getNamespaceResources();
+
+        if (!cr.clusterExists(cluster)) {
+            cr.createCluster(cluster,
+                    ClusterData.builder()
+                            .serviceUrl(this.getWebServiceAddress())
+                            .serviceUrlTls(this.getWebServiceAddressTls())
+                            .brokerServiceUrl(this.getBrokerServiceUrl())
+                            .brokerServiceUrlTls(this.getBrokerServiceUrlTls())
+                            .build());
+        }
+
+        if (!tr.tenantExists(publicTenant)) {
+            tr.createTenant(publicTenant,
+                    TenantInfo.builder()
+                            .adminRoles(Sets.newHashSet(config.getSuperUserRoles()))
+                            .allowedClusters(Sets.newHashSet(cluster))
+                            .build());
+        }
+
+        if (!nsr.namespaceExists(ns)) {
+            Policies nsp = new Policies();

Review Comment:
   Don't we need to create the namespace before?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java:
##########
@@ -58,6 +61,15 @@ public interface LoadManager {
      */
     Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception;
 
+    default CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return null;

Review Comment:
   Please throw `UnsupportedOperationException` for the default functions. 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java:
##########
@@ -58,6 +61,15 @@ public interface LoadManager {
      */
     Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception;
 
+    default CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return null;
+    }
+
+    default CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return null;

Review Comment:
   Please throw `UnsupportedOperationException` for the default functions. 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {

Review Comment:
   Though this `start()` call is not expected from multiple threads on the same obj. This atomic get() check can still allow multiple threads to `start()` on this obj.
   We can simply make this `synchronized` because we don't expect start() to be called competitively from many threads.
   Or, if we want to use the atomic boolean, we can use CAS here.
   ```
   if(started.compareAndSet(false, true)){
     try{
       // start logic here
     } catch ( Exception e){
      started.set(false) // to allow retry.
    }
   }
   ```
   
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java:
##########
@@ -0,0 +1,222 @@
+/*

Review Comment:
   I assume this class does not need a review here, as it is added in a separate PR, https://github.com/apache/pulsar/pull/18810.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {

Review Comment:
   https://github.com/apache/pulsar/pull/18964 has been merged. We could use LeastResourceUsageWithWeight here. 
   
   Also, we need a TODO: comment here to make brokerSelectionStrategy configurable.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.
+    }
+
+    @Override
+    public void writeLoadReportOnZookeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.
+    }
+
+    @Override
+    public void writeLoadReportOnZookeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public void writeResourceQuotasToZooKeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java:
##########
@@ -118,7 +118,7 @@ public interface ServiceUnitStateChannel extends Closeable {
      *                 the future object will time out.
      * Case 3: If none of them, it returns null.

Review Comment:
   With this change, the case 3 returns Optional.empty(). Please update the comment.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -242,11 +242,12 @@ public CompletableFuture<Optional<String>> getChannelOwnerAsync() {
                     // TODO: discard this protocol prefix removal
                     //  by a util func that returns lookupServiceAddress(serviceUrl)
                     if (leader.isPresent()) {
-                        String broker = leader.get().getServiceUrl();
-                        broker = broker.substring(broker.lastIndexOf('/') + 1);
-                        return Optional.of(broker);
+                        return Optional.of(leader.get().getServiceUrl());
                     } else {
-                        return Optional.empty();

Review Comment:
   If we don't return Optional.empty(), we can change the output signature to  `CompletableFuture<String>`.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java:
##########
@@ -0,0 +1,313 @@
+/*

Review Comment:
   I assume this class does not need a review here, as it is added in a separate PR, https://github.com/apache/pulsar/pull/18810.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -171,11 +172,21 @@ public void initialize() {
         }
     }
 
+    public boolean isExtensibleLoadManager(){

Review Comment:
   Please use the static func, isExtensibleLoadManagerEnabled(), as discussed above.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java:
##########
@@ -132,7 +132,7 @@ public interface ServiceUnitStateChannel extends Closeable {
      * case 2: If the assigned broker does not take the ownership in time,
      *         the future object will time out.
      */
-    CompletableFuture<String> publishAssignEventAsync(String serviceUnit, String broker);
+    CompletableFuture<Optional<String>> publishAssignEventAsync(String serviceUnit, String broker);

Review Comment:
   Why do we need Optional here? I don't think we return Optional.empty here.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java:
##########
@@ -19,6 +19,7 @@
 package org.apache.pulsar.broker.loadbalance.extensions;

Review Comment:
   I assume this class does not need a review here, as it is added in a separate PR, https://github.com/apache/pulsar/pull/18810.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;

Review Comment:
   plz add an error log.
   
   Why don't we throw an exception here or return the empty broker?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);

Review Comment:
   nit: discover(select)  -> select



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java:
##########
@@ -143,6 +155,11 @@ static LoadManager create(final PulsarService pulsar) {
                 final LoadManager casted = new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance);
                 casted.initialize(pulsar);
                 return casted;
+            } else if (loadManagerInstance instanceof ExtensibleLoadManagerImpl) {

Review Comment:
   nit : can we compare against the interface, `(loadManagerInstance instanceof ExtensibleLoadManager) ` to be consistent with the other cases?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.
+    }
+
+    @Override
+    public void writeLoadReportOnZookeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public void writeResourceQuotasToZooKeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public List<Metrics> getLoadBalancingMetrics() {
+        return null;

Review Comment:
   Add a TODO comment here.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());
+
+                    return CompletableFuture.completedFuture(
+                            getBrokerSelectionStrategy().select(candidateBrokers, bundle, context));
+                });
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundleUnit) {
+        final String bundle = bundleUnit.toString();
+        CompletableFuture<Optional<String>> owner;
+        if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+            owner = serviceUnitStateChannel.getChannelOwnerAsync();
+        } else {
+            owner = serviceUnitStateChannel.getOwnerAsync(bundle);
+        }
+
+        return owner.thenApply(broker -> brokerRegistry.getBrokerId().equals(broker.orElse(null)));
+    }
+
+    @Override
+    public void close() throws PulsarServerException {

Review Comment:
   Like `start()` this can still allow multiple threads can `close()` on this obj.
   I think we can simply make this synchronized too.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.
+    }
+
+    @Override
+    public void writeLoadReportOnZookeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public void writeResourceQuotasToZooKeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public List<Metrics> getLoadBalancingMetrics() {
+        return null;
+    }
+
+    @Override
+    public void doLoadShedding() {
+        // No-op.

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -171,11 +172,21 @@ public void initialize() {
         }
     }
 
+    public boolean isExtensibleLoadManager(){
+        return loadManager.get() instanceof ExtensibleLoadManagerWrapper;
+    }
+
     public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicName topic, LookupOptions options) {
         long startTime = System.nanoTime();
 
         CompletableFuture<Optional<LookupResult>> future = getBundleAsync(topic)
-                .thenCompose(bundle -> findBrokerServiceUrl(bundle, options));
+                .thenCompose(bundle -> {
+                    if (isExtensibleLoadManager()) {

Review Comment:
   I was working on these NamespaceService change as per `pluggable new broker load balancer` work.
   
   Do you want to keep this change here in this PR?
   
   I think we need to add NamespaceService unit tests to track this variation.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java:
##########
@@ -242,11 +242,12 @@ public CompletableFuture<Optional<String>> getChannelOwnerAsync() {
                     // TODO: discard this protocol prefix removal
                     //  by a util func that returns lookupServiceAddress(serviceUrl)
                     if (leader.isPresent()) {
-                        String broker = leader.get().getServiceUrl();
-                        broker = broker.substring(broker.lastIndexOf('/') + 1);
-                        return Optional.of(broker);
+                        return Optional.of(leader.get().getServiceUrl());

Review Comment:
   Do we keep the prefix here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   Isn't availableBrokerCandidates already copied?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.loadbalance.LoadManager;
+import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.lookup.LookupResult;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.stats.Metrics;
+import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
+
+public class ExtensibleLoadManagerWrapper implements LoadManager {
+
+    private PulsarService pulsar;
+
+    private final ExtensibleLoadManagerImpl loadManager;
+
+    public ExtensibleLoadManagerWrapper(ExtensibleLoadManagerImpl loadManager) {
+        this.loadManager = loadManager;
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        loadManager.start();
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        loadManager.initialize(pulsar);
+        this.pulsar = pulsar;
+    }
+
+    @Override
+    public boolean isCentralized() {
+        return true;
+    }
+
+    @Override
+    public CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
+            Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.assign(topic, bundle)
+                .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
+        return loadManager.checkOwnershipAsync(topic, bundle);
+    }
+
+    @Override
+    public void disableBroker() throws Exception {
+        this.loadManager.getBrokerRegistry().unregister();
+    }
+
+    @Override
+    public Set<String> getAvailableBrokers() throws Exception {
+        return getAvailableBrokersAsync()
+                .get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> getAvailableBrokersAsync() {
+        return this.loadManager.getBrokerRegistry().getAvailableBrokersAsync().thenApply(HashSet::new);
+    }
+
+    @Override
+    public void stop() throws PulsarServerException {
+        this.loadManager.close();
+    }
+
+
+    @Override
+    public Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception {
+        return Optional.empty();
+    }
+
+    @Override
+    public LoadManagerReport generateLoadReport() {
+        return null;
+    }
+
+    @Override
+    public void setLoadReportForceUpdateFlag() {
+        // No-op.
+    }
+
+    @Override
+    public void writeLoadReportOnZookeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public void writeResourceQuotasToZooKeeper() throws Exception {
+        // No-op, this operation is not useful, the load data reporter will automatically write.
+    }
+
+    @Override
+    public List<Metrics> getLoadBalancingMetrics() {
+        return null;
+    }
+
+    @Override
+    public void doLoadShedding() {
+        // No-op.
+    }
+
+    @Override
+    public void doNamespaceBundleSplit() {
+        // No-op.

Review Comment:
   Throw UnsupportedOperationException.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);

Review Comment:
   Isn't availableBrokers already copied from the cache?



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061321963


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java:
##########
@@ -51,7 +51,7 @@ public interface ServiceUnitStateChannel extends Closeable {
      *
      * ServiceUnitStateChannel elects the separate leader as the owner broker of the system topic in this channel.
      */
-    CompletableFuture<Optional<String>> getChannelOwnerAsync();
+    CompletableFuture<String> getChannelOwnerAsync();

Review Comment:
   Good point. Changed back.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061331104


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }

Review Comment:
   Throwing an unchecked exception in the callback of `CompletableFuture` is not a good behavior. The caller usually prints the stack of the exception (in `exceptionally` or `whenComplete`). But the stack is meaningless. A better practice is to print an error log here and return an empty `Optional`.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061372645


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }

Review Comment:
   The caller handled this exception, see: https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java#L315-L317
   
   And in the old logic, it also throws the `IllegalStateException`, https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java#L398-L400
   
   



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


Re: [PR] [improve][broker] PIP-192: Implement extensible load manager [pulsar]

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1493927408


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -175,7 +176,14 @@ public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicN
         long startTime = System.nanoTime();
 
         CompletableFuture<Optional<LookupResult>> future = getBundleAsync(topic)
-                .thenCompose(bundle -> findBrokerServiceUrl(bundle, options));
+                .thenCompose(bundle -> {
+                    if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) {
+                        return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle);
+                    } else {
+                        // TODO: Add unit tests cover it.

Review Comment:
   Thanks for your advice. We should refactor the NamespaceService instead of adding so many if else here. We can remove this to-do when we're doing refactoring.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059216341


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {

Review Comment:
   It also depends on the https://github.com/apache/pulsar/pull/18777 .



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1060228467


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -171,11 +172,21 @@ public void initialize() {
         }
     }
 
+    public boolean isExtensibleLoadManager(){
+        return loadManager.get() instanceof ExtensibleLoadManagerWrapper;
+    }
+
     public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicName topic, LookupOptions options) {
         long startTime = System.nanoTime();
 
         CompletableFuture<Optional<LookupResult>> future = getBundleAsync(topic)
-                .thenCompose(bundle -> findBrokerServiceUrl(bundle, options));
+                .thenCompose(bundle -> {
+                    if (isExtensibleLoadManager()) {

Review Comment:
   I'll add the units test to cover enable the extensible load manager case.



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


[GitHub] [pulsar] lhotari commented on pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "lhotari (via GitHub)" <gi...@apache.org>.
lhotari commented on PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#issuecomment-1495413728

   This PR introduced a new flaky test #20007 . @Demogorgon314 Do you have a chance to fix that?


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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "BewareMyPower (via GitHub)" <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1093994357


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    public static final String BROKER_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-broker-load-data").toString();
+
+    public static final String TOP_BUNDLES_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-top-bundles-load-data").toString();
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    private LoadDataStore<BrokerLoadData> brokerLoadDataStore;
+    private LoadDataStore<TopBundlesLoadData> topBundlesLoadDataStore;

Review Comment:
   These two fields are only used in `start()`, will them be used somewhere else in future? If not, we should change them to local variables.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1093986856


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java:
##########
@@ -0,0 +1,222 @@
+/*

Review Comment:
   The PR https://github.com/apache/pulsar/pull/18810 is merged. We can continue the review now : )



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "BewareMyPower (via GitHub)" <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1094021602


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException;
+import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    public static final String BROKER_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-broker-load-data").toString();
+
+    public static final String TOP_BUNDLES_LOAD_DATA_STORE_TOPIC = TopicName.get(
+            TopicDomain.non_persistent.value(),
+            NamespaceName.SYSTEM_NAMESPACE,
+            "loadbalancer-top-bundles-load-data").toString();
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    private LoadDataStore<BrokerLoadData> brokerLoadDataStore;
+    private LoadDataStore<TopBundlesLoadData> topBundlesLoadDataStore;

Review Comment:
   Just a question, other code LGTM, so I approved it first.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059521507


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   `ArrayList<String> candidateBrokers = availableBrokerCandidates.keySet();`
   
   Can we just use keySet() without copying?



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061209983


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;

Review Comment:
   I have changed to return the empty broker, since the exception does not happen frequently.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061321237


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java:
##########
@@ -51,7 +51,7 @@ public interface ServiceUnitStateChannel extends Closeable {
      *
      * ServiceUnitStateChannel elects the separate leader as the owner broker of the system topic in this channel.
      */
-    CompletableFuture<Optional<String>> getChannelOwnerAsync();
+    CompletableFuture<String> getChannelOwnerAsync();

Review Comment:
   Using an `Optional` could make code more simple. e.g. the `getChannelOwnerAsync` can be implemented like:
   
   ```java
           return leaderElectionService.readCurrentLeader().thenApply(leader -> leader.map(LeaderBroker::getServiceUrl)
                           .map(url -> url.substring(url.lastIndexOf('/') + 1)));
   ```
   
   `isChannelOwnerAsync`:
   
   ```java
   return getChannelOwnerAsync().thenApply(optOwner -> optOwner.map(this::isTargetBroker).orElse(false));
   ```
   
   If you didn't return an `Optional`, the caller side might mix the expected error (i.e. the leader is absent) and unexpected errors (i.e. NPE or other unchecked exceptions).



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061341875


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   @heesung-sn Maybe we can change the parameter type from `List<String>` to `Collections<String>` in `BrokerSelectionStrategy`? If we want to access the item by index, we can create a copy manually. If we only need to iterate over the collection, we don't have to copy the items.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   @heesung-sn Maybe we can change the parameter type from `List<String>` to `Collection<String>` in `BrokerSelectionStrategy`? If we want to access the item by index, we can create a copy manually. If we only need to iterate over the collection, we don't have to copy the items.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059231154


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;

Review Comment:
   Do we need to keep the same behavior as the original(`ModularLoadManagerImpl`)? When has an exception, it will restore the list of brokers to the full set



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061333762


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }

Review Comment:
   For example, in this method, the caller side only needs to handle if the assign operation succeeded. If you throw an exception here, the caller side should process it like:
   
   ```java
   if (e instanceof IllegalStateException) {
       log.error("Failed to assign: {}", e.getMessage());
   } else {
       log.error("Unexpected exception in assign", e);
   }
   ```



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1093033235


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   Fixed, I changed to `Set<String>`.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059245743


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -925,6 +939,36 @@ public void start() throws PulsarServerException {
         }
     }
 
+    protected void createNamespaceIfNotExists(String cluster, String publicTenant, NamespaceName ns) throws Exception {
+        ClusterResources cr = this.getPulsarResources().getClusterResources();
+        TenantResources tr = this.getPulsarResources().getTenantResources();
+        NamespaceResources nsr = this.getPulsarResources().getNamespaceResources();
+
+        if (!cr.clusterExists(cluster)) {
+            cr.createCluster(cluster,
+                    ClusterData.builder()
+                            .serviceUrl(this.getWebServiceAddress())
+                            .serviceUrlTls(this.getWebServiceAddressTls())
+                            .brokerServiceUrl(this.getBrokerServiceUrl())
+                            .brokerServiceUrlTls(this.getBrokerServiceUrlTls())
+                            .build());
+        }
+
+        if (!tr.tenantExists(publicTenant)) {
+            tr.createTenant(publicTenant,
+                    TenantInfo.builder()
+                            .adminRoles(Sets.newHashSet(config.getSuperUserRoles()))
+                            .allowedClusters(Sets.newHashSet(cluster))
+                            .build());
+        }
+
+        if (!nsr.namespaceExists(ns)) {
+            Policies nsp = new Policies();

Review Comment:
   This is how to create the namespace.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1068833055


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;

Review Comment:
   plz add a TODO: comment here. We may need to revisit this error case. 
   
   For now, let's stick to the original logic.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059229901


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -794,6 +802,12 @@ public void start() throws PulsarServerException {
             }
             brokerService.start();
 
+            if (this.loadManager.get() instanceof ExtensibleLoadManagerWrapper) {

Review Comment:
   Updated.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061194276


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   You can see the `BrokerSelectionStrategy` use `List<String>` as the candidate brokers.
   
   ```java
       Optional<String> select(List<String> brokers, ServiceUnitId bundle, LoadManagerContext context);
   ```



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


[GitHub] [pulsar] codecov-commenter commented on pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#issuecomment-1415164108

   # [Codecov](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#19102](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3684d5c) into [master](https://codecov.io/gh/apache/pulsar/commit/60d8e645e7ca6afa46625b15f32c1b65862888ce?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (60d8e64) will **decrease** coverage by `0.76%`.
   > The diff coverage is `8.69%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pulsar/pull/19102/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #19102      +/-   ##
   ============================================
   - Coverage     63.24%   62.49%   -0.76%     
   + Complexity    25936    25660     -276     
   ============================================
     Files          1823     1829       +6     
     Lines        133338   133915     +577     
     Branches      14677    14732      +55     
   ============================================
   - Hits          84330    83690     -640     
   - Misses        41290    42565    +1275     
   + Partials       7718     7660      -58     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | inttests | `24.85% <8.69%> (-0.04%)` | :arrow_down: |
   | systests | `25.68% <8.69%> (+0.01%)` | :arrow_up: |
   | unittests | `59.72% <8.69%> (-0.91%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../apache/pulsar/broker/loadbalance/LoadManager.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9Mb2FkTWFuYWdlci5qYXZh) | `58.33% <0.00%> (-19.45%)` | :arrow_down: |
   | [...lance/extensions/ExtensibleLoadManagerWrapper.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9leHRlbnNpb25zL0V4dGVuc2libGVMb2FkTWFuYWdlcldyYXBwZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...loadbalance/extensions/LoadManagerContextImpl.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9leHRlbnNpb25zL0xvYWRNYW5hZ2VyQ29udGV4dEltcGwuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...xtensions/channel/ServiceUnitStateChannelImpl.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9leHRlbnNpb25zL2NoYW5uZWwvU2VydmljZVVuaXRTdGF0ZUNoYW5uZWxJbXBsLmphdmE=) | `0.96% <0.00%> (-0.01%)` | :arrow_down: |
   | [...ensions/strategy/LeastResourceUsageWithWeight.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9leHRlbnNpb25zL3N0cmF0ZWd5L0xlYXN0UmVzb3VyY2VVc2FnZVdpdGhXZWlnaHQuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...dbalance/extensions/ExtensibleLoadManagerImpl.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9sb2FkYmFsYW5jZS9leHRlbnNpb25zL0V4dGVuc2libGVMb2FkTWFuYWdlckltcGwuamF2YQ==) | `7.54% <7.54%> (ø)` | |
   | [...ache/pulsar/broker/namespace/NamespaceService.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9uYW1lc3BhY2UvTmFtZXNwYWNlU2VydmljZS5qYXZh) | `59.14% <23.07%> (-2.06%)` | :arrow_down: |
   | [...n/java/org/apache/pulsar/broker/PulsarService.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9QdWxzYXJTZXJ2aWNlLmphdmE=) | `73.87% <33.33%> (+1.10%)` | :arrow_up: |
   | [...ava/org/apache/pulsar/broker/stats/WindowWrap.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zdGF0cy9XaW5kb3dXcmFwLmphdmE=) | `0.00% <0.00%> (-84.62%)` | :arrow_down: |
   | [...er/metadata/v2/TransactionBufferSnapshotIndex.java](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci90cmFuc2FjdGlvbi9idWZmZXIvbWV0YWRhdGEvdjIvVHJhbnNhY3Rpb25CdWZmZXJTbmFwc2hvdEluZGV4LmphdmE=) | `0.00% <0.00%> (-77.78%)` | :arrow_down: |
   | ... and [158 more](https://codecov.io/gh/apache/pulsar/pull/19102?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1066087400


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   nit: Actually, we can have a separate PR to integrate ExtensibleLoadManagerImpl into the existing code(to the NamespaceService). This PR could be ExtensibleLoadManagerImply only.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061504336


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }

Review Comment:
   The exception is handled for `NamespaceService#getBrokerServiceUrlAsync`, are they equivalent?
   
   BTW, you also throws an `IllegalStateException` in line 136, which means `broker.isEmpty()` is always false here. Even if `ExtensibleLoadManager#assign` is equivalent and will replace `NamespaceService#getBrokerServiceUrlAsync`, the line 136 should return an empty `Optional` to go to the if branch of line 147.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059522860


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   It would be great to add unit tests in NamespaceServiceTest for these if-else variations. (assert if the new/old load manager is called for these Namespace public funcs).
   
   Or, please add a TODO comment if we want to add such tests later.
   ```
   if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) {
   // new logic
   } else {
   // old logic
   }
   ```



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061350534


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokerCandidates -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            log.error("Failed to filter out brokers.", e);
+                            return CompletableFuture.completedFuture(Optional.empty());
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());
+
+                    return CompletableFuture.completedFuture(
+                            getBrokerSelectionStrategy().select(candidateBrokers, bundle, context));
+                });
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundleUnit) {
+        final String bundle = bundleUnit.toString();
+        CompletableFuture<Optional<String>> owner;
+        if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+            owner = serviceUnitStateChannel.getChannelOwnerAsync();
+        } else {
+            owner = serviceUnitStateChannel.getOwnerAsync(bundle);
+        }
+
+        return owner.thenApply(broker -> brokerRegistry.getBrokerId().equals(broker.orElse(null)));
+    }
+
+    @Override
+    public synchronized void close() throws PulsarServerException {
+        if (!this.started) {
+            return;
+        }
+        try {
+            this.brokerRegistry.close();
+        } catch (Exception e) {
+            throw new PulsarServerException(e);
+        }
+        this.serviceUnitStateChannel.close();
+        this.started = false;

Review Comment:
   Move it to `finally` because when `serviceUnitStateChannel.close()` throws an `PulsarServerException`, it will be skipped.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061252680


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   After thinking about it, I guess adding tests later might be better since some methods of `NamespaceService` still need to add new logic when using extensible load manager.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061363555


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -925,6 +939,36 @@ public void start() throws PulsarServerException {
         }
     }
 
+    protected void createNamespaceIfNotExists(String cluster, String publicTenant, NamespaceName ns) throws Exception {
+        ClusterResources cr = this.getPulsarResources().getClusterResources();
+        TenantResources tr = this.getPulsarResources().getTenantResources();
+        NamespaceResources nsr = this.getPulsarResources().getNamespaceResources();
+
+        if (!cr.clusterExists(cluster)) {
+            cr.createCluster(cluster,
+                    ClusterData.builder()
+                            .serviceUrl(this.getWebServiceAddress())
+                            .serviceUrlTls(this.getWebServiceAddressTls())
+                            .brokerServiceUrl(this.getBrokerServiceUrl())
+                            .brokerServiceUrlTls(this.getBrokerServiceUrlTls())
+                            .build());
+        }
+
+        if (!tr.tenantExists(publicTenant)) {
+            tr.createTenant(publicTenant,
+                    TenantInfo.builder()
+                            .adminRoles(Sets.newHashSet(config.getSuperUserRoles()))
+                            .allowedClusters(Sets.newHashSet(cluster))
+                            .build());
+        }
+
+        if (!nsr.namespaceExists(ns)) {
+            Policies nsp = new Policies();

Review Comment:
   I agree with @heesung-sn that we should create the namespace in advance. We create namespaces in standalone because the standalone mode is deployed simply. But brokers can start concurrently, so we should create the namespace in advance in https://pulsar.apache.org/docs/2.10.x/deploy-bare-metal/#initialize-cluster-metadata



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "BewareMyPower (via GitHub)" <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1092725182


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java:
##########
@@ -0,0 +1,222 @@
+/*

Review Comment:
   And we can review and merge #18810 first since `ExtensibleLoadManagerImpl` depends on the `BrokerRegistry`.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059231314


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   The `BrokerSelectionStrategy` requires a `List` instead of set.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   The `BrokerSelectionStrategy` requires a `List` instead of `Set`.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1066087400


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   nit: Actually, we can have a separate PR to integrate ExtensibleLoadManagerImpl into the existing code(to the NamespaceService and PulsarService). This PR could be ExtensibleLoadManagerImply only.



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1068832278


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -55,6 +55,7 @@
 import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
 import org.apache.pulsar.broker.loadbalance.LoadManager;
 import org.apache.pulsar.broker.loadbalance.ResourceUnit;
+import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;

Review Comment:
   I tried to split it into two PRs, but `ExtensibleLoadManagerImpl ` test dependency topic lookup because we are using `TableView` in `ServiceUnitStateChannel`, so I have to implement the load manager function.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1066084076


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   The collection should be fine too.



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


[GitHub] [pulsar] gaoran10 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "gaoran10 (via GitHub)" <gi...@apache.org>.
gaoran10 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1092767910


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();

Review Comment:
   Do we need to use the modifier synchronized?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java:
##########
@@ -0,0 +1,222 @@
+/*

Review Comment:
   +1, we'd better merge the PR #18810 first.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());

Review Comment:
   Maybe we can change the parameter type from`List<String>` to `Set<String>` for the method `BrokerSelectionStrategy#select`? The type of the collection is `String`, it shouldn't have repeated brokers.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "BewareMyPower (via GitHub)" <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1092723582


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java:
##########
@@ -0,0 +1,222 @@
+/*

Review Comment:
   Could you remove the `BrokerRegistryImpl` related code in this PR since I see they were added in #18810?



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


[GitHub] [pulsar] Demogorgon314 commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
Demogorgon314 commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061193599


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);

Review Comment:
   It uses to recover the candidate when the filter has an exception.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061313426


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java:
##########
@@ -51,7 +51,7 @@ public interface ServiceUnitStateChannel extends Closeable {
      *
      * ServiceUnitStateChannel elects the separate leader as the owner broker of the system topic in this channel.
      */
-    CompletableFuture<Optional<String>> getChannelOwnerAsync();
+    CompletableFuture<String> getChannelOwnerAsync();

Review Comment:
   We should keep using the `Optional` here. I see the comments [here](https://github.com/apache/pulsar/pull/19102/files#r1059201582). But the root cause is that you should not throw an unchecked exception when leader is empty. It's an expected error and we should pass the result (leader is empty, i.e. an empty `Optional` is returned) to the caller side.



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


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1061354952


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private final List<BrokerFilter> brokerFilterPipeline;
+
+    private boolean started = false;
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        // TODO: Make brokerSelectionStrategy configurable.
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
+        return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
+    }
+
+    @Override
+    public synchronized void start() throws PulsarServerException {
+        if (this.started) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+        this.started = true;
+    }
+
+    @Override
+    public synchronized void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get())
+                                        .thenApply(Optional::of);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to select the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokerCandidates -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            log.error("Failed to filter out brokers.", e);
+                            return CompletableFuture.completedFuture(Optional.empty());
+                        }
+                    }
+                    if (availableBrokerCandidates.isEmpty()) {
+                        return CompletableFuture.completedFuture(Optional.empty());
+                    }
+                    ArrayList<String> candidateBrokers = new ArrayList<>(availableBrokerCandidates.keySet());
+
+                    return CompletableFuture.completedFuture(
+                            getBrokerSelectionStrategy().select(candidateBrokers, bundle, context));
+                });
+    }
+
+    @Override
+    public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundleUnit) {
+        final String bundle = bundleUnit.toString();
+        CompletableFuture<Optional<String>> owner;
+        if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+            owner = serviceUnitStateChannel.getChannelOwnerAsync();
+        } else {
+            owner = serviceUnitStateChannel.getOwnerAsync(bundle);
+        }
+
+        return owner.thenApply(broker -> brokerRegistry.getBrokerId().equals(broker.orElse(null)));
+    }
+
+    @Override
+    public synchronized void close() throws PulsarServerException {
+        if (!this.started) {
+            return;
+        }
+        try {
+            this.brokerRegistry.close();
+        } catch (Exception e) {
+            throw new PulsarServerException(e);
+        }
+        this.serviceUnitStateChannel.close();
+        this.started = false;

Review Comment:
   Additional, `serviceUnitStateChannel.close()` could be skipped when `brokerRegistry.close()` throws an exception (BTW, why did you catch `Exception`?)
   
   If you don't care which exception will be thrown, you can simply write like:
   
   ```java
           try {
               this.brokerRegistry.close();
           } finally {
               try {
                   this.serviceUnitStateChannel.close();
               } finally {
                   this.started = false;
               }
           }
   ```
   
   Otherwise, it would be a little complicated to combine two `PulsarServerException` into a single `PulsarServerException`.



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


[GitHub] [pulsar] heesung-sn commented on a diff in pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by GitBox <gi...@apache.org>.
heesung-sn commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1059521330


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.loadbalance.extensions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
+import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
+import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
+import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
+import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
+import org.apache.pulsar.common.naming.ServiceUnitId;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+
+@Slf4j
+public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
+
+    private PulsarService pulsar;
+
+    private ServiceConfiguration conf;
+
+    @Getter
+    private BrokerRegistry brokerRegistry;
+
+    private ServiceUnitStateChannel serviceUnitStateChannel;
+
+    @Getter
+    private LoadManagerContext context;
+
+    @Getter
+    private final BrokerSelectionStrategy brokerSelectionStrategy;
+
+    @Getter
+    private List<BrokerFilter> brokerFilterPipeline;
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+
+    private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<BrokerLookupData>>>
+            lookupRequests = ConcurrentOpenHashMap.<String,
+                    CompletableFuture<Optional<BrokerLookupData>>>newBuilder()
+            .build();
+
+    /**
+     * Life cycle: Constructor -> initialize -> start -> close.
+     */
+    public ExtensibleLoadManagerImpl() {
+        this.brokerFilterPipeline = new ArrayList<>();
+        this.brokerSelectionStrategy = (brokers, bundle, context) -> {
+            if (brokers.isEmpty()) {
+                return Optional.empty();
+            }
+            return Optional.of(brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())));
+        };
+    }
+
+    @Override
+    public void start() throws PulsarServerException {
+        if (this.started.get()) {
+            return;
+        }
+        this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+        this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
+        this.brokerRegistry.start();
+        this.serviceUnitStateChannel.start();
+
+        // TODO: Start the load data store.
+
+        this.context = LoadManagerContextImpl.builder()
+                .configuration(conf)
+                .brokerRegistry(brokerRegistry)
+                .brokerLoadDataStore(null)
+                .topBundleLoadDataStore(null).build();
+        // TODO: Start load data reporter.
+
+        // TODO: Start unload scheduler and bundle split scheduler
+
+        this.started.set(true);
+    }
+
+    @Override
+    public void initialize(PulsarService pulsar) {
+        this.pulsar = pulsar;
+        this.conf = pulsar.getConfiguration();
+    }
+
+    @Override
+    public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
+                                                                ServiceUnitId serviceUnit) {
+
+        final String bundle = serviceUnit.toString();
+
+        CompletableFuture<Optional<BrokerLookupData>> future = lookupRequests.computeIfAbsent(bundle, k -> {
+            final CompletableFuture<Optional<String>> owner;
+            // Assign the bundle to channel owner if is internal topic, to avoid circular references.
+            if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
+                owner = serviceUnitStateChannel.getChannelOwnerAsync();
+            } else {
+                owner = serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
+                    // If the bundle not assign yet, select and publish assign event to channel.
+                    if (broker.isEmpty()) {
+                        return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
+                            if (brokerOpt.isPresent()) {
+                                log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
+                                return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get());
+                            } else {
+                                throw new IllegalStateException(
+                                        "Failed to discover(select) the new owner broker for bundle: " + bundle);
+                            }
+                        });
+                    }
+                    // Already assigned, return it.
+                    return CompletableFuture.completedFuture(broker);
+                });
+            }
+
+            return owner.thenCompose(broker -> {
+                if (broker.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(broker.get());
+            }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> {
+                if (brokerLookupData.isEmpty()) {
+                    String errorMsg = String.format(
+                            "Failed to look up a broker registry:%s for bundle:%s", broker, bundle);
+                    log.error(errorMsg);
+                    throw new IllegalStateException(errorMsg);
+                }
+                return CompletableFuture.completedFuture(brokerLookupData);
+            }));
+        });
+        future.whenComplete((r, t) -> lookupRequests.remove(bundle));
+        return future;
+    }
+
+    public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {
+        BrokerRegistry brokerRegistry = getBrokerRegistry();
+        return brokerRegistry.getAvailableBrokerLookupDataAsync()
+                .thenCompose(availableBrokers -> {
+                    // TODO: Support isolation policies
+                    LoadManagerContext context = this.getContext();
+
+                    // Filter out brokers that do not meet the rules.
+                    List<BrokerFilter> filterPipeline = getBrokerFilterPipeline();
+                    Map<String, BrokerLookupData> availableBrokerCandidates = new HashMap<>(availableBrokers);
+                    for (final BrokerFilter filter : filterPipeline) {
+                        try {
+                            filter.filter(availableBrokerCandidates, context);
+                        } catch (BrokerFilterException e) {
+                            availableBrokerCandidates = availableBrokers;

Review Comment:
   What if the version filter throws an exception, and this `selectAsync` returns an old version broker? Is this acceptable?



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


[GitHub] [pulsar] Demogorgon314 merged pull request #19102: [improve][broker] PIP-192: Implement extensible load manager

Posted by "Demogorgon314 (via GitHub)" <gi...@apache.org>.
Demogorgon314 merged PR #19102:
URL: https://github.com/apache/pulsar/pull/19102


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


Re: [PR] [improve][broker] PIP-192: Implement extensible load manager [pulsar]

Posted by "asafm (via GitHub)" <gi...@apache.org>.
asafm commented on code in PR #19102:
URL: https://github.com/apache/pulsar/pull/19102#discussion_r1493786444


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java:
##########
@@ -175,7 +176,14 @@ public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicN
         long startTime = System.nanoTime();
 
         CompletableFuture<Optional<LookupResult>> future = getBundleAsync(topic)
-                .thenCompose(bundle -> findBrokerServiceUrl(bundle, options));
+                .thenCompose(bundle -> {
+                    if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) {
+                        return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle);
+                    } else {
+                        // TODO: Add unit tests cover it.

Review Comment:
   My personal habit is to never add those TODOs - they never get done. 



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