heesung-sn commented on code in PR #18810: URL: https://github.com/apache/pulsar/pull/18810#discussion_r1047897296
########## pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java: ########## @@ -0,0 +1,302 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +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.extensions.data.BrokerLookupData; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.coordination.LockManager; +import org.apache.pulsar.metadata.api.coordination.ResourceLock; + +/** + * The broker registry impl, base on the LockManager. + */ +@Slf4j +public class BrokerRegistryImpl implements BrokerRegistry { + + private static final String LOOKUP_DATA_PATH = "/loadbalance/brokers"; + + private final PulsarService pulsar; + + private final ServiceConfiguration conf; + + private final BrokerLookupData brokerLookupData; + + private final LockManager<BrokerLookupData> brokerLookupDataLockManager; + + private final String brokerZNodePath; + + private final String lookupServiceAddress; + + @VisibleForTesting + protected final Map<String, BrokerLookupData> brokerLookupDataCache; + + private final ScheduledExecutorService scheduler; + + private final List<BiConsumer<String, NotificationType>> listeners; + + private volatile ResourceLock<BrokerLookupData> brokerLookupDataLock; + + private volatile boolean registered = false; + + private final AtomicBoolean started = new AtomicBoolean(false); + + private volatile CompletableFuture<Void> cacheReloadFuture; + + public BrokerRegistryImpl(PulsarService pulsar) { + this.pulsar = pulsar; + this.conf = pulsar.getConfiguration(); + this.brokerLookupDataLockManager = pulsar.getCoordinationService().getLockManager(BrokerLookupData.class); + this.scheduler = pulsar.getLoadManagerExecutor(); + this.brokerLookupDataCache = new ConcurrentHashMap<>(); + this.listeners = new ArrayList<>(); + this.cacheReloadFuture = CompletableFuture.completedFuture(null); + this.lookupServiceAddress = pulsar.getLookupServiceAddress(); + this.brokerZNodePath = LOOKUP_DATA_PATH + "/" + lookupServiceAddress; + this.brokerLookupData = new BrokerLookupData( + pulsar.getSafeWebServiceAddress(), + pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), + pulsar.getBrokerServiceUrlTls(), + pulsar.getAdvertisedListeners(), + pulsar.getProtocolDataToAdvertise(), + pulsar.getConfiguration().isEnablePersistentTopics(), + pulsar.getConfiguration().isEnableNonPersistentTopics(), + pulsar.getBrokerVersion()); + } + + @Override + public void start() throws PulsarServerException { + if (started.get()) { + return; + } + pulsar.getLocalMetadataStore().registerListener(this::handleMetadataStoreNotification); + try { + this.register(); + // Update all lookup data to cache + this.reloadAllBrokerLookupCacheAsync() + .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + this.started.set(true); + } catch (MetadataStoreException | ExecutionException | InterruptedException | TimeoutException e) { + throw new PulsarServerException(e); + } + } + + @Override + public boolean isStarted() { + return this.started.get(); + } + + @Override + public synchronized void register() throws MetadataStoreException { + if (!registered) { + try { + this.brokerLookupDataLock = brokerLookupDataLockManager.acquireLock(brokerZNodePath, brokerLookupData) + .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + registered = true; + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw MetadataStoreException.unwrap(e); + } + } + } + + @Override + public synchronized void unregister() throws MetadataStoreException { + if (registered) { + try { + brokerLookupDataLock.release().get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + registered = false; + } catch (CompletionException | InterruptedException | ExecutionException | TimeoutException e) { + throw MetadataStoreException.unwrap(e); + } + } + } + + @Override + public String getLookupServiceAddress() { Review Comment: I think getBrokerId() is fine too. ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java: ########## @@ -0,0 +1,218 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +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.extensions.data.BrokerLookupData; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.coordination.LockManager; +import org.apache.pulsar.metadata.api.coordination.ResourceLock; + +/** + * The broker registry impl, base on the LockManager. + */ +@Slf4j +public class BrokerRegistryImpl implements BrokerRegistry { + + private static final String LOOKUP_DATA_PATH = "/loadbalance/brokers"; + + private final PulsarService pulsar; + + private final ServiceConfiguration conf; + + private final BrokerLookupData brokerLookupData; + + private final LockManager<BrokerLookupData> brokerLookupDataLockManager; + + private final String brokerZNodePath; + + private final String lookupServiceAddress; + + @VisibleForTesting + protected final Map<String, BrokerLookupData> brokerLookupDataMap; + + private final ScheduledExecutorService scheduler; + + private final List<BiConsumer<String, NotificationType>> listeners; + + private final AtomicBoolean registered; + + private volatile ResourceLock<BrokerLookupData> brokerLookupDataLock; + + public BrokerRegistryImpl(PulsarService pulsar) { + this.pulsar = pulsar; + this.conf = pulsar.getConfiguration(); + this.brokerLookupDataLockManager = pulsar.getCoordinationService().getLockManager(BrokerLookupData.class); + this.scheduler = pulsar.getLoadManagerExecutor(); + this.brokerLookupDataMap = new ConcurrentHashMap<>(); + this.listeners = new ArrayList<>(); + + this.registered = new AtomicBoolean(false); + this.lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + + conf.getWebServicePort().orElseGet(() -> conf.getWebServicePortTls().get()); + this.brokerZNodePath = LOOKUP_DATA_PATH + "/" + lookupServiceAddress; + this.brokerLookupData = new BrokerLookupData( + pulsar.getSafeWebServiceAddress(), + pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), + pulsar.getBrokerServiceUrlTls(), + pulsar.getAdvertisedListeners(), + pulsar.getProtocolDataToAdvertise(), + pulsar.getConfiguration().isEnablePersistentTopics(), + pulsar.getConfiguration().isEnableNonPersistentTopics(), + pulsar.getBrokerVersion()); + } + + @Override + public void start() { + pulsar.getLocalMetadataStore().registerListener(this::handleMetadataStoreNotification); + } + + @Override + public void register() { + if (registered.compareAndSet(false, true)) { + this.brokerLookupDataLock = + brokerLookupDataLockManager.acquireLock(brokerZNodePath, brokerLookupData).join(); + } + } + + @Override + public void unregister() throws MetadataStoreException { + if (registered.compareAndSet(true, false)) { + try { + brokerLookupDataLock.release().join(); + } catch (CompletionException e) { + throw MetadataStoreException.unwrap(e); + } + } + } + + @Override + public String getLookupServiceAddress() { + return this.lookupServiceAddress; + } + + @Override + public CompletableFuture<List<String>> getAvailableBrokersAsync() { + CompletableFuture<List<String>> future = new CompletableFuture<>(); + brokerLookupDataLockManager.listLocks(LOOKUP_DATA_PATH) Review Comment: I see that we maintain an internal cache with ttl inside LockManagerImpl. It appears that we don't need to maintain `brokerLookupDataMap` separately. Can you clarify why we need `brokerLookupDataMap`? ref: ``` @Override public CompletableFuture<List<String>> listLocks(String path) { return cache.getChildren(path); } @Override public CompletableFuture<List<String>> getChildren(String path) { return store.getChildren(path); } @Override public final CompletableFuture<List<String>> getChildren(String path) { if (!isValidPath(path)) { return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); } return childrenCache.get(path); } this.childrenCache = Caffeine.newBuilder() .refreshAfterWrite(CACHE_REFRESH_TIME_MILLIS, TimeUnit.MILLISECONDS) .expireAfterWrite(CACHE_REFRESH_TIME_MILLIS * 2, TimeUnit.MILLISECONDS) .buildAsync(new AsyncCacheLoader<String, List<String>>() { @Override public CompletableFuture<List<String>> asyncLoad(String key, Executor executor) { return getChildrenFromStore(key); } @Override public CompletableFuture<List<String>> asyncReload(String key, List<String> oldValue, Executor executor) { if (isConnected) { return getChildrenFromStore(key); } else { // Do not refresh if we're not connected return CompletableFuture.completedFuture(oldValue); } } }); ``` Also, the broker delete notification is already handled by ``` private void handleDataNotification(Notification n) { if (n.getType() == NotificationType.Deleted) { ResourceLockImpl<T> lock = locks.get(n.getPath()); if (lock != null) { lock.lockWasInvalidated(); } } } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
