listener) {
this.leaderElection = cs.getLeaderElection(LeaderBroker.class, electionRoot, listener);
- this.localValue = new LeaderBroker(localWebServiceAddress);
+ this.localValue = new LeaderBroker(brokerId, serviceUrl);
}
public void start() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java
index 17aa7170fc63c..61f34ef4901ba 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java
@@ -161,7 +161,8 @@ public static long getCpuUsageForCGroup() {
*
*
* Line is split in "words", filtering the first. The sum of all numbers give the amount of cpu cycles used this
- * far. Real CPU usage should equal the sum subtracting the idle cycles, this would include iowait, irq and steal.
+ * far. Real CPU usage should equal the sum substracting the idle cycles(that is idle+iowait), this would include
+ * cpu, user, nice, system, irq, softirq, steal, guest and guest_nice.
*/
public static ResourceUsage getCpuUsageForEntireHost() {
try (Stream stream = Files.lines(Paths.get(PROC_STAT_PATH))) {
@@ -175,7 +176,7 @@ public static ResourceUsage getCpuUsageForEntireHost() {
.filter(s -> !s.contains("cpu"))
.mapToLong(Long::parseLong)
.sum();
- long idle = Long.parseLong(words[4]);
+ long idle = Long.parseLong(words[4]) + Long.parseLong(words[5]);
return ResourceUsage.builder()
.usage(total - idle)
.idle(idle)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java
index a632a47f05116..c1fe2a4930c34 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java
@@ -64,7 +64,7 @@ public Map getBundleData() {
public Map getBundleDataForLoadShedding() {
return bundleData.entrySet().stream()
- .filter(e -> !NamespaceService.filterNamespaceForShedding(
+ .filter(e -> !NamespaceService.isSLAOrHeartbeatNamespace(
NamespaceBundle.getBundleNamespace(e.getKey())))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java
index 0de2ae92db61a..f9f36b705d4c4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/NoopLoadManager.java
@@ -43,7 +43,7 @@
public class NoopLoadManager implements LoadManager {
private PulsarService pulsar;
- private String lookupServiceAddress;
+ private String brokerId;
private ResourceUnit localResourceUnit;
private LockManager lockManager;
private Map bundleBrokerAffinityMap;
@@ -57,16 +57,15 @@ public void initialize(PulsarService pulsar) {
@Override
public void start() throws PulsarServerException {
- lookupServiceAddress = pulsar.getLookupServiceAddress();
- localResourceUnit = new SimpleResourceUnit(String.format("http://%s", lookupServiceAddress),
- new PulsarResourceDescription());
+ brokerId = pulsar.getBrokerId();
+ localResourceUnit = new SimpleResourceUnit(brokerId, new PulsarResourceDescription());
- LocalBrokerData localData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(),
+ LocalBrokerData localData = new LocalBrokerData(pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners());
localData.setProtocols(pulsar.getProtocolDataToAdvertise());
localData.setLoadManagerClassName(this.pulsar.getConfig().getLoadManagerClassName());
- String brokerReportPath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress;
+ String brokerReportPath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + brokerId;
try {
log.info("Acquiring broker resource lock on {}", brokerReportPath);
@@ -129,12 +128,12 @@ public void disableBroker() throws Exception {
@Override
public Set getAvailableBrokers() throws Exception {
- return Collections.singleton(lookupServiceAddress);
+ return Collections.singleton(brokerId);
}
@Override
public CompletableFuture> getAvailableBrokersAsync() {
- return CompletableFuture.completedFuture(Collections.singleton(lookupServiceAddress));
+ return CompletableFuture.completedFuture(Collections.singleton(brokerId));
}
@Override
@@ -153,7 +152,6 @@ public String setNamespaceBundleAffinity(String bundle, String broker) {
if (StringUtils.isBlank(broker)) {
return this.bundleBrokerAffinityMap.remove(bundle);
}
- broker = broker.replaceFirst("http[s]?://", "");
return this.bundleBrokerAffinityMap.put(bundle, broker);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/ResourceUnit.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/ResourceUnit.java
index ef4dd2a97b280..c28a8be4c0d3a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/ResourceUnit.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/ResourceUnit.java
@@ -23,8 +23,6 @@
*/
public interface ResourceUnit extends Comparable {
- String PROPERTY_KEY_BROKER_ZNODE_NAME = "__advertised_addr";
-
String getResourceId();
ResourceDescription getAvailableResource();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java
index 921ce35b5c65e..18e30ddf922d0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java
@@ -82,9 +82,9 @@ public BrokerRegistryImpl(PulsarService pulsar) {
this.brokerLookupDataLockManager = pulsar.getCoordinationService().getLockManager(BrokerLookupData.class);
this.scheduler = pulsar.getLoadManagerExecutor();
this.listeners = new ArrayList<>();
- this.brokerId = pulsar.getLookupServiceAddress();
+ this.brokerId = pulsar.getBrokerId();
this.brokerLookupData = new BrokerLookupData(
- pulsar.getSafeWebServiceAddress(),
+ pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(),
pulsar.getBrokerServiceUrlTls(),
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java
index cba499eb8eedb..6a0e677c66268 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java
@@ -21,6 +21,7 @@
import static java.lang.String.format;
import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Follower;
import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Leader;
+import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.TOPIC;
import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Label.Success;
import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin;
import static org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared.getNamespaceBundle;
@@ -44,6 +45,7 @@
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
@@ -86,6 +88,7 @@
import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared;
import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies;
import org.apache.pulsar.broker.namespace.NamespaceEphemeralData;
+import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm;
@@ -95,7 +98,6 @@
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.common.util.FutureUtil;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.apache.pulsar.metadata.api.coordination.LeaderElectionState;
import org.slf4j.Logger;
@@ -116,6 +118,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
private static final long MONITOR_INTERVAL_IN_MILLIS = 120_000;
+ public static final long COMPACTION_THRESHOLD = 5 * 1024 * 1024;
+
private static final String ELECTION_ROOT = "/loadbalance/extension/leader";
private PulsarService pulsar;
@@ -152,6 +156,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
@Getter
private final List brokerFilterPipeline;
+
/**
* The load data reporter.
*/
@@ -171,6 +176,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
private volatile boolean started = false;
+ private boolean configuredSystemTopics = false;
+
private final AssignCounter assignCounter = new AssignCounter();
@Getter
private final UnloadCounter unloadCounter = new UnloadCounter();
@@ -181,10 +188,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
// record split metrics
private final AtomicReference> splitMetrics = new AtomicReference<>();
- private final ConcurrentOpenHashMap>>
- lookupRequests = ConcurrentOpenHashMap.>>newBuilder()
- .build();
+ private final ConcurrentHashMap>>
+ lookupRequests = new ConcurrentHashMap<>();
private final CountDownLatch initWaiter = new CountDownLatch(1);
/**
@@ -197,7 +202,7 @@ public Set getOwnedServiceUnits() {
}
Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet();
String brokerId = brokerRegistry.getBrokerId();
- return entrySet.stream()
+ Set ownedServiceUnits = entrySet.stream()
.filter(entry -> {
var stateData = entry.getValue();
return stateData.state() == ServiceUnitState.Owned
@@ -207,6 +212,36 @@ public Set getOwnedServiceUnits() {
var bundle = entry.getKey();
return getNamespaceBundle(pulsar, bundle);
}).collect(Collectors.toSet());
+ // Add heartbeat and SLA monitor namespace bundle.
+ NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration());
+ try {
+ NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory()
+ .getFullBundle(heartbeatNamespace);
+ ownedServiceUnits.add(fullBundle);
+ } catch (Exception e) {
+ log.warn("Failed to get heartbeat namespace bundle.", e);
+ }
+ NamespaceName heartbeatNamespaceV2 = NamespaceService
+ .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration());
+ try {
+ NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory()
+ .getFullBundle(heartbeatNamespaceV2);
+ ownedServiceUnits.add(fullBundle);
+ } catch (Exception e) {
+ log.warn("Failed to get heartbeat namespace V2 bundle.", e);
+ }
+
+ NamespaceName slaMonitorNamespace = NamespaceService
+ .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration());
+ try {
+ NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory()
+ .getFullBundle(slaMonitorNamespace);
+ ownedServiceUnits.add(fullBundle);
+ } catch (Exception e) {
+ log.warn("Failed to get SLA Monitor namespace bundle.", e);
+ }
+
+ return ownedServiceUnits;
}
public enum Role {
@@ -232,6 +267,10 @@ public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
}
+ public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) {
+ return pulsar.getLoadManager().get() instanceof ExtensibleLoadManagerWrapper;
+ }
+
public static ExtensibleLoadManagerImpl get(LoadManager loadManager) {
if (!(loadManager instanceof ExtensibleLoadManagerWrapper loadManagerWrapper)) {
throw new IllegalArgumentException("The load manager should be 'ExtensibleLoadManagerWrapper'.");
@@ -249,114 +288,145 @@ public static void createSystemTopic(PulsarService pulsar, String topic) throws
log.info("Created topic {}.", topic);
} catch (PulsarAdminException.ConflictException ex) {
if (debug(pulsar.getConfiguration(), log)) {
- log.info("Topic {} already exists.", topic, ex);
+ log.info("Topic {} already exists.", topic);
}
} catch (PulsarAdminException e) {
throw new PulsarServerException(e);
}
}
+ private static void createSystemTopics(PulsarService pulsar) throws PulsarServerException {
+ createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC);
+ createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC);
+ }
+
+ private static boolean configureSystemTopics(PulsarService pulsar) {
+ try {
+ if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)
+ && (pulsar.getConfiguration().isSystemTopicEnabled()
+ && pulsar.getConfiguration().isTopicLevelPoliciesEnabled())) {
+ Long threshold = pulsar.getAdminClient().topicPolicies().getCompactionThreshold(TOPIC);
+ if (threshold == null || COMPACTION_THRESHOLD != threshold.longValue()) {
+ pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, COMPACTION_THRESHOLD);
+ log.info("Set compaction threshold: {} bytes for system topic {}.", COMPACTION_THRESHOLD, TOPIC);
+ }
+ } else {
+ log.warn("System topic or topic level policies is disabled. "
+ + "{} compaction threshold follows the broker or namespace policies.", TOPIC);
+ }
+ return true;
+ } catch (Exception e) {
+ log.error("Failed to set compaction threshold for system topic:{}", TOPIC, e);
+ }
+ return false;
+ }
+
@Override
public void start() throws PulsarServerException {
if (this.started) {
return;
}
- this.brokerRegistry = new BrokerRegistryImpl(pulsar);
- this.leaderElectionService = new LeaderElectionService(
- pulsar.getCoordinationService(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT,
- state -> {
- pulsar.getLoadManagerExecutor().execute(() -> {
- if (state == LeaderElectionState.Leading) {
- playLeader();
- } else {
- playFollower();
- }
+ try {
+ this.brokerRegistry = new BrokerRegistryImpl(pulsar);
+ this.leaderElectionService = new LeaderElectionService(
+ pulsar.getCoordinationService(), pulsar.getBrokerId(),
+ pulsar.getSafeWebServiceAddress(), ELECTION_ROOT,
+ state -> {
+ pulsar.getLoadManagerExecutor().execute(() -> {
+ if (state == LeaderElectionState.Leading) {
+ playLeader();
+ } else {
+ playFollower();
+ }
+ });
});
- });
- this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
- this.brokerRegistry.start();
- this.splitManager = new SplitManager(splitCounter);
- this.unloadManager = new UnloadManager(unloadCounter);
- this.serviceUnitStateChannel.listen(unloadManager);
- this.serviceUnitStateChannel.listen(splitManager);
- this.leaderElectionService.start();
- this.serviceUnitStateChannel.start();
- this.antiAffinityGroupPolicyHelper =
- new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel);
- antiAffinityGroupPolicyHelper.listenFailureDomainUpdate();
- this.antiAffinityGroupPolicyFilter = new AntiAffinityGroupPolicyFilter(antiAffinityGroupPolicyHelper);
- this.brokerFilterPipeline.add(antiAffinityGroupPolicyFilter);
- SimpleResourceAllocationPolicies policies = new SimpleResourceAllocationPolicies(pulsar);
- this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies);
- this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper));
+ this.serviceUnitStateChannel = ServiceUnitStateChannelImpl.newInstance(pulsar);
+ this.brokerRegistry.start();
+ this.splitManager = new SplitManager(splitCounter);
+ this.unloadManager = new UnloadManager(unloadCounter);
+ this.serviceUnitStateChannel.listen(unloadManager);
+ this.serviceUnitStateChannel.listen(splitManager);
+ this.leaderElectionService.start();
+ this.serviceUnitStateChannel.start();
+ this.antiAffinityGroupPolicyHelper =
+ new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel);
+ antiAffinityGroupPolicyHelper.listenFailureDomainUpdate();
+ this.antiAffinityGroupPolicyFilter = new AntiAffinityGroupPolicyFilter(antiAffinityGroupPolicyHelper);
+ this.brokerFilterPipeline.add(antiAffinityGroupPolicyFilter);
+ SimpleResourceAllocationPolicies policies = new SimpleResourceAllocationPolicies(pulsar);
+ this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies);
+ this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper));
- createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC);
- createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC);
+ try {
+ this.brokerLoadDataStore = LoadDataStoreFactory
+ .create(pulsar, BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class);
+ this.topBundlesLoadDataStore = LoadDataStoreFactory
+ .create(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class);
+ } catch (LoadDataStoreException e) {
+ throw new PulsarServerException(e);
+ }
- try {
- this.brokerLoadDataStore = LoadDataStoreFactory
- .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class);
- this.brokerLoadDataStore.startTableView();
- this.topBundlesLoadDataStore = LoadDataStoreFactory
- .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class);
- } catch (LoadDataStoreException e) {
- throw new PulsarServerException(e);
+ this.context = LoadManagerContextImpl.builder()
+ .configuration(conf)
+ .brokerRegistry(brokerRegistry)
+ .brokerLoadDataStore(brokerLoadDataStore)
+ .topBundleLoadDataStore(topBundlesLoadDataStore).build();
+
+ this.brokerLoadDataReporter =
+ new BrokerLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), brokerLoadDataStore);
+
+ this.topBundleLoadDataReporter =
+ new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore);
+ this.serviceUnitStateChannel.listen(brokerLoadDataReporter);
+ this.serviceUnitStateChannel.listen(topBundleLoadDataReporter);
+ var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis();
+ this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
+ .scheduleAtFixedRate(() -> {
+ try {
+ brokerLoadDataReporter.reportAsync(false);
+ // TODO: update broker load metrics using getLocalData
+ } catch (Throwable e) {
+ log.error("Failed to run the broker load manager executor job.", e);
+ }
+ },
+ interval,
+ interval, TimeUnit.MILLISECONDS);
+
+ this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
+ .scheduleAtFixedRate(() -> {
+ try {
+ // TODO: consider excluding the bundles that are in the process of split.
+ topBundleLoadDataReporter.reportAsync(false);
+ } catch (Throwable e) {
+ log.error("Failed to run the top bundles load manager executor job.", e);
+ }
+ },
+ interval,
+ interval, TimeUnit.MILLISECONDS);
+
+ this.monitorTask = this.pulsar.getLoadManagerExecutor()
+ .scheduleAtFixedRate(() -> {
+ monitor();
+ },
+ MONITOR_INTERVAL_IN_MILLIS,
+ MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS);
+
+ this.unloadScheduler = new UnloadScheduler(
+ pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context,
+ serviceUnitStateChannel, unloadCounter, unloadMetrics);
+ this.splitScheduler = new SplitScheduler(
+ pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context);
+ this.splitScheduler.start();
+ this.initWaiter.countDown();
+ this.started = true;
+ log.info("Started load manager.");
+ } catch (Exception ex) {
+ log.error("Failed to start the extensible load balance and close broker registry {}.",
+ this.brokerRegistry, ex);
+ if (this.brokerRegistry != null) {
+ brokerRegistry.close();
+ }
}
-
- this.context = LoadManagerContextImpl.builder()
- .configuration(conf)
- .brokerRegistry(brokerRegistry)
- .brokerLoadDataStore(brokerLoadDataStore)
- .topBundleLoadDataStore(topBundlesLoadDataStore).build();
-
- this.brokerLoadDataReporter =
- new BrokerLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), brokerLoadDataStore);
-
- this.topBundleLoadDataReporter =
- new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore);
- this.serviceUnitStateChannel.listen(brokerLoadDataReporter);
- this.serviceUnitStateChannel.listen(topBundleLoadDataReporter);
- var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis();
- this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
- .scheduleAtFixedRate(() -> {
- try {
- brokerLoadDataReporter.reportAsync(false);
- // TODO: update broker load metrics using getLocalData
- } catch (Throwable e) {
- log.error("Failed to run the broker load manager executor job.", e);
- }
- },
- interval,
- interval, TimeUnit.MILLISECONDS);
-
- this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
- .scheduleAtFixedRate(() -> {
- try {
- // TODO: consider excluding the bundles that are in the process of split.
- topBundleLoadDataReporter.reportAsync(false);
- } catch (Throwable e) {
- log.error("Failed to run the top bundles load manager executor job.", e);
- }
- },
- interval,
- interval, TimeUnit.MILLISECONDS);
-
- this.monitorTask = this.pulsar.getLoadManagerExecutor()
- .scheduleAtFixedRate(() -> {
- monitor();
- },
- MONITOR_INTERVAL_IN_MILLIS,
- MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS);
-
- this.unloadScheduler = new UnloadScheduler(
- pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context,
- serviceUnitStateChannel, unloadCounter, unloadMetrics);
- this.unloadScheduler.start();
- this.splitScheduler = new SplitScheduler(
- pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context);
- this.splitScheduler.start();
- this.initWaiter.countDown();
- this.started = true;
}
@Override
@@ -377,25 +447,38 @@ public CompletableFuture> assign(Optional getOwnerAsync(
- ServiceUnitId serviceUnit, String bundle, boolean ownByLocalBrokerIfAbsent) {
+ private String getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) {
+ // Check if this is Heartbeat or SLAMonitor namespace
+ String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit);
+ if (candidateBroker == null) {
+ candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit);
+ }
+ if (candidateBroker == null) {
+ candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit);
+ }
+ if (candidateBroker != null) {
+ return candidateBroker.substring(candidateBroker.lastIndexOf('/') + 1);
+ }
+ return candidateBroker;
+ }
+
+ private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit,
+ String bundle) {
return serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> {
// If the bundle not assign yet, select and publish assign event to channel.
if (broker.isEmpty()) {
- CompletableFuture> selectedBroker;
- if (ownByLocalBrokerIfAbsent) {
- String brokerId = this.brokerRegistry.getBrokerId();
- selectedBroker = CompletableFuture.completedFuture(Optional.of(brokerId));
- } else {
- selectedBroker = this.selectAsync(serviceUnit);
- }
- return selectedBroker.thenCompose(brokerOpt -> {
+ return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> {
if (brokerOpt.isPresent()) {
assignCounter.incrementSuccess();
log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle);
@@ -425,7 +508,8 @@ private CompletableFuture> getBrokerLookupData(
}).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);
+ "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.",
+ broker, bundle);
log.error(errorMsg);
throw new IllegalStateException(errorMsg);
}
@@ -443,30 +527,37 @@ private CompletableFuture> getBrokerLookupData(
public CompletableFuture tryAcquiringOwnership(NamespaceBundle namespaceBundle) {
log.info("Try acquiring ownership for bundle: {} - {}.", namespaceBundle, brokerRegistry.getBrokerId());
final String bundle = namespaceBundle.toString();
- return dedupeLookupRequest(bundle, k -> {
- final CompletableFuture owner =
- this.getOwnerAsync(namespaceBundle, bundle, true);
- return getBrokerLookupData(owner.thenApply(Optional::ofNullable), bundle);
- }).thenApply(brokerLookupData -> {
- if (brokerLookupData.isEmpty()) {
- throw new IllegalStateException(
- "Failed to get the broker lookup data for bundle: " + bundle);
- }
- return brokerLookupData.get().toNamespaceEphemeralData();
- });
+ return assign(Optional.empty(), namespaceBundle)
+ .thenApply(brokerLookupData -> {
+ if (brokerLookupData.isEmpty()) {
+ String errorMsg = String.format(
+ "Failed to get the broker lookup data for bundle:%s", bundle);
+ log.error(errorMsg);
+ throw new IllegalStateException(errorMsg);
+ }
+ return brokerLookupData.get().toNamespaceEphemeralData();
+ });
}
private CompletableFuture> dedupeLookupRequest(
String key, Function>> provider) {
- CompletableFuture> future = lookupRequests.computeIfAbsent(key, provider);
- future.whenComplete((r, t) -> {
- if (t != null) {
+ final MutableObject>> newFutureCreated = new MutableObject<>();
+ try {
+ return lookupRequests.computeIfAbsent(key, k -> {
+ CompletableFuture> future = provider.apply(k);
+ newFutureCreated.setValue(future);
+ return future;
+ });
+ } finally {
+ if (newFutureCreated.getValue() != null) {
+ newFutureCreated.getValue().whenComplete((v, ex) -> {
+ if (ex != null) {
assignCounter.incrementFailure();
}
lookupRequests.remove(key);
- }
- );
- return future;
+ });
+ }
+ }
}
public CompletableFuture> selectAsync(ServiceUnitId bundle) {
@@ -521,15 +612,16 @@ public CompletableFuture checkOwnershipAsync(Optional to
}
public CompletableFuture> getOwnershipAsync(Optional topic,
- ServiceUnitId bundleUnit) {
- final String bundle = bundleUnit.toString();
- CompletableFuture> owner;
+ ServiceUnitId serviceUnit) {
+ final String bundle = serviceUnit.toString();
if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
- owner = serviceUnitStateChannel.getChannelOwnerAsync();
- } else {
- owner = serviceUnitStateChannel.getOwnerAsync(bundle);
+ return serviceUnitStateChannel.getChannelOwnerAsync();
+ }
+ String candidateBroker = getHeartbeatOrSLAMonitorBrokerId(serviceUnit);
+ if (candidateBroker != null) {
+ return CompletableFuture.completedFuture(Optional.of(candidateBroker));
}
- return owner;
+ return serviceUnitStateChannel.getOwnerAsync(bundle);
}
public CompletableFuture> getOwnershipWithLookupDataAsync(ServiceUnitId bundleUnit) {
@@ -543,6 +635,10 @@ public CompletableFuture> getOwnershipWithLookupDataA
public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle,
Optional destinationBroker) {
+ if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) {
+ log.info("Skip unloading namespace bundle: {}.", bundle);
+ return CompletableFuture.completedFuture(null);
+ }
return getOwnershipAsync(Optional.empty(), bundle)
.thenCompose(brokerOpt -> {
if (brokerOpt.isEmpty()) {
@@ -577,6 +673,10 @@ private CompletableFuture unloadAsync(UnloadDecision unloadDecision,
public CompletableFuture splitNamespaceBundleAsync(ServiceUnitId bundle,
NamespaceBundleSplitAlgorithm splitAlgorithm,
List boundaries) {
+ if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) {
+ log.info("Skip split namespace bundle: {}.", bundle);
+ return CompletableFuture.completedFuture(null);
+ }
final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle.toString());
final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle.toString());
NamespaceBundle namespaceBundle =
@@ -666,82 +766,106 @@ public void close() throws PulsarServerException {
}
}
- private boolean isInternalTopic(String topic) {
- return topic.startsWith(ServiceUnitStateChannelImpl.TOPIC)
+ public static boolean isInternalTopic(String topic) {
+ return topic.startsWith(TOPIC)
|| topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC)
|| topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC);
}
@VisibleForTesting
- void playLeader() {
- if (role != Leader) {
- log.info("This broker:{} is changing the role from {} to {}",
- pulsar.getLookupServiceAddress(), role, Leader);
- int retry = 0;
- while (true) {
- try {
- initWaiter.await();
- serviceUnitStateChannel.scheduleOwnershipMonitor();
- topBundlesLoadDataStore.startTableView();
- unloadScheduler.start();
+ synchronized void playLeader() {
+ log.info("This broker:{} is setting the role from {} to {}",
+ pulsar.getBrokerId(), role, Leader);
+ int retry = 0;
+ boolean becameFollower = false;
+ while (!Thread.currentThread().isInterrupted()) {
+ try {
+ if (!serviceUnitStateChannel.isChannelOwner()) {
+ becameFollower = true;
break;
- } catch (Throwable e) {
- log.error("The broker:{} failed to change the role. Retrying {} th ...",
- pulsar.getLookupServiceAddress(), ++retry, e);
- try {
- Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS));
- } catch (InterruptedException ex) {
- log.warn("Interrupted while sleeping.");
- }
+ }
+ initWaiter.await();
+ // Confirm the system topics have been created or create them if they do not exist.
+ // If the leader has changed, the new leader need to reset
+ // the local brokerService.topics (by this topic creations).
+ // Otherwise, the system topic existence check will fail on the leader broker.
+ createSystemTopics(pulsar);
+ brokerLoadDataStore.init();
+ topBundlesLoadDataStore.init();
+ unloadScheduler.start();
+ serviceUnitStateChannel.scheduleOwnershipMonitor();
+ break;
+ } catch (Throwable e) {
+ log.warn("The broker:{} failed to set the role. Retrying {} th ...",
+ pulsar.getBrokerId(), ++retry, e);
+ try {
+ Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS));
+ } catch (InterruptedException ex) {
+ log.warn("Interrupted while sleeping.");
+ // preserve thread's interrupt status
+ Thread.currentThread().interrupt();
}
}
- role = Leader;
- log.info("This broker:{} plays the leader now.", pulsar.getLookupServiceAddress());
}
- // flush the load data when the leader is elected.
- if (brokerLoadDataReporter != null) {
- brokerLoadDataReporter.reportAsync(true);
- }
- if (topBundleLoadDataReporter != null) {
- topBundleLoadDataReporter.reportAsync(true);
+ if (becameFollower) {
+ log.warn("The broker:{} became follower while initializing leader role.", pulsar.getBrokerId());
+ playFollower();
+ return;
}
+
+ role = Leader;
+ log.info("This broker:{} plays the leader now.", pulsar.getBrokerId());
+
+ // flush the load data when the leader is elected.
+ brokerLoadDataReporter.reportAsync(true);
+ topBundleLoadDataReporter.reportAsync(true);
}
@VisibleForTesting
- void playFollower() {
- if (role != Follower) {
- log.info("This broker:{} is changing the role from {} to {}",
- pulsar.getLookupServiceAddress(), role, Follower);
- int retry = 0;
- while (true) {
- try {
- initWaiter.await();
- serviceUnitStateChannel.cancelOwnershipMonitor();
- topBundlesLoadDataStore.closeTableView();
- unloadScheduler.close();
+ synchronized void playFollower() {
+ log.info("This broker:{} is setting the role from {} to {}",
+ pulsar.getBrokerId(), role, Follower);
+ int retry = 0;
+ boolean becameLeader = false;
+ while (!Thread.currentThread().isInterrupted()) {
+ try {
+ if (serviceUnitStateChannel.isChannelOwner()) {
+ becameLeader = true;
break;
- } catch (Throwable e) {
- log.error("The broker:{} failed to change the role. Retrying {} th ...",
- pulsar.getLookupServiceAddress(), ++retry, e);
- try {
- Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS));
- } catch (InterruptedException ex) {
- log.warn("Interrupted while sleeping.");
- }
+ }
+ initWaiter.await();
+ unloadScheduler.close();
+ serviceUnitStateChannel.cancelOwnershipMonitor();
+ brokerLoadDataStore.init();
+ topBundlesLoadDataStore.close();
+ topBundlesLoadDataStore.startProducer();
+ break;
+ } catch (Throwable e) {
+ log.warn("The broker:{} failed to set the role. Retrying {} th ...",
+ pulsar.getBrokerId(), ++retry, e);
+ try {
+ Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS));
+ } catch (InterruptedException ex) {
+ log.warn("Interrupted while sleeping.");
+ // preserve thread's interrupt status
+ Thread.currentThread().interrupt();
}
}
- role = Follower;
- log.info("This broker:{} plays a follower now.", pulsar.getLookupServiceAddress());
}
- // flush the load data when the leader is elected.
- if (brokerLoadDataReporter != null) {
- brokerLoadDataReporter.reportAsync(true);
- }
- if (topBundleLoadDataReporter != null) {
- topBundleLoadDataReporter.reportAsync(true);
+ if (becameLeader) {
+ log.warn("This broker:{} became leader while initializing follower role.", pulsar.getBrokerId());
+ playLeader();
+ return;
}
+
+ role = Follower;
+ log.info("This broker:{} plays a follower now.", pulsar.getBrokerId());
+
+ // flush the load data when the leader is elected.
+ brokerLoadDataReporter.reportAsync(true);
+ topBundleLoadDataReporter.reportAsync(true);
}
public List getMetrics() {
@@ -765,7 +889,9 @@ public List getMetrics() {
return metricsCollection;
}
- private void monitor() {
+
+ @VisibleForTesting
+ protected void monitor() {
try {
initWaiter.await();
@@ -773,6 +899,11 @@ private void monitor() {
// Periodically check the role in case ZK watcher fails.
var isChannelOwner = serviceUnitStateChannel.isChannelOwner();
if (isChannelOwner) {
+ // System topic config might fail due to the race condition
+ // with topic policy init(Topic policies cache have not init).
+ if (!configuredSystemTopics) {
+ configuredSystemTopics = configureSystemTopics(pulsar);
+ }
if (role != Leader) {
log.warn("Current role:{} does not match with the channel ownership:{}. "
+ "Playing the leader role.", role, isChannelOwner);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java
index 18e949537dedb..cd1561cb70e2d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java
@@ -118,13 +118,11 @@ public void setLoadReportForceUpdateFlag() {
@Override
public void writeLoadReportOnZookeeper() throws Exception {
// No-op, this operation is not useful, the load data reporter will automatically write.
- throw new UnsupportedOperationException();
}
@Override
public void writeResourceQuotasToZooKeeper() throws Exception {
// No-op, this operation is not useful, the load data reporter will automatically write.
- throw new UnsupportedOperationException();
}
@Override
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java
index 717ff484fe772..1471d4a75c175 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java
@@ -54,6 +54,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledFuture;
@@ -67,10 +68,12 @@
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.pulsar.PulsarClusterMetadataSetup;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.loadbalance.LeaderBroker;
import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistry;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;
@@ -97,7 +100,6 @@
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.common.topics.TopicCompactionStrategy;
import org.apache.pulsar.common.util.FutureUtil;
-import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.NotificationType;
import org.apache.pulsar.metadata.api.extended.SessionEvent;
@@ -125,9 +127,9 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel {
private final PulsarService pulsar;
private final ServiceConfiguration config;
private final Schema schema;
- private final ConcurrentOpenHashMap> getOwnerRequests;
- private final String lookupServiceAddress;
- private final ConcurrentOpenHashMap> cleanupJobs;
+ private final Map> getOwnerRequests;
+ private final String brokerId;
+ private final Map> cleanupJobs;
private final StateChangeListeners stateChangeListeners;
private ExtensibleLoadManagerImpl loadManager;
private BrokerRegistry brokerRegistry;
@@ -199,19 +201,29 @@ enum MetadataState {
Unstable
}
+ public static ServiceUnitStateChannelImpl newInstance(PulsarService pulsar) {
+ return new ServiceUnitStateChannelImpl(pulsar);
+ }
+
public ServiceUnitStateChannelImpl(PulsarService pulsar) {
+ this(pulsar, MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS, OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS);
+ }
+
+ @VisibleForTesting
+ public ServiceUnitStateChannelImpl(PulsarService pulsar,
+ long inFlightStateWaitingTimeInMillis,
+ long ownershipMonitorDelayTimeInSecs) {
this.pulsar = pulsar;
this.config = pulsar.getConfig();
- this.lookupServiceAddress = pulsar.getLookupServiceAddress();
+ this.brokerId = pulsar.getBrokerId();
this.schema = Schema.JSON(ServiceUnitStateData.class);
- this.getOwnerRequests = ConcurrentOpenHashMap.>newBuilder().build();
- this.cleanupJobs = ConcurrentOpenHashMap.>newBuilder().build();
+ this.getOwnerRequests = new ConcurrentHashMap<>();
+ this.cleanupJobs = new ConcurrentHashMap<>();
this.stateChangeListeners = new StateChangeListeners();
this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds()
* 1000;
- this.inFlightStateWaitingTimeInMillis = MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS;
- this.ownershipMonitorDelayTimeInSecs = OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS;
+ this.inFlightStateWaitingTimeInMillis = inFlightStateWaitingTimeInMillis;
+ this.ownershipMonitorDelayTimeInSecs = ownershipMonitorDelayTimeInSecs;
if (semiTerminalStateWaitingTimeInMillis < inFlightStateWaitingTimeInMillis) {
throw new IllegalArgumentException(
"Invalid Config: loadBalancerServiceUnitStateCleanUpDelayTimeInSeconds < "
@@ -249,7 +261,7 @@ public void scheduleOwnershipMonitor() {
},
0, ownershipMonitorDelayTimeInSecs, SECONDS);
log.info("This leader broker:{} started the ownership monitor.",
- lookupServiceAddress);
+ brokerId);
}
}
@@ -258,13 +270,13 @@ public void cancelOwnershipMonitor() {
monitorTask.cancel(false);
monitorTask = null;
log.info("This previous leader broker:{} stopped the ownership monitor.",
- lookupServiceAddress);
+ brokerId);
}
}
@Override
public void cleanOwnerships() {
- doCleanup(lookupServiceAddress);
+ doCleanup(brokerId);
}
public synchronized void start() throws PulsarServerException {
@@ -430,19 +442,8 @@ public CompletableFuture> getChannelOwnerAsync() {
new IllegalStateException("Invalid channel state:" + channelState.name()));
}
- return leaderElectionService.readCurrentLeader().thenApply(leader -> {
- //expecting http://broker-xyz:port
- // 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);
- } else {
- return Optional.empty();
- }
- }
- );
+ return leaderElectionService.readCurrentLeader()
+ .thenApply(leader -> leader.map(LeaderBroker::getBrokerId));
}
public CompletableFuture isChannelOwnerAsync() {
@@ -484,7 +485,35 @@ public boolean isOwner(String serviceUnit, String targetBroker) {
}
public boolean isOwner(String serviceUnit) {
- return isOwner(serviceUnit, lookupServiceAddress);
+ return isOwner(serviceUnit, brokerId);
+ }
+
+ private CompletableFuture> getActiveOwnerAsync(
+ String serviceUnit,
+ ServiceUnitState state,
+ Optional owner) {
+ return deferGetOwnerRequest(serviceUnit)
+ .thenCompose(newOwner -> {
+ if (newOwner == null) {
+ return CompletableFuture.completedFuture(null);
+ }
+
+ return brokerRegistry.lookupAsync(newOwner)
+ .thenApply(lookupData -> {
+ if (lookupData.isPresent()) {
+ return newOwner;
+ } else {
+ throw new IllegalStateException(
+ "The new owner " + newOwner + " is inactive.");
+ }
+ });
+ }).whenComplete((__, e) -> {
+ if (e != null) {
+ log.error("{} failed to get active owner broker. serviceUnit:{}, state:{}, owner:{}",
+ brokerId, serviceUnit, state, owner, e);
+ ownerLookUpCounters.get(state).getFailure().incrementAndGet();
+ }
+ }).thenApply(Optional::ofNullable);
}
public CompletableFuture> getOwnerAsync(String serviceUnit) {
@@ -498,18 +527,22 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) {
ownerLookUpCounters.get(state).getTotal().incrementAndGet();
switch (state) {
case Owned -> {
- return CompletableFuture.completedFuture(Optional.of(data.dstBroker()));
+ return getActiveOwnerAsync(serviceUnit, state, Optional.of(data.dstBroker()));
}
case Splitting -> {
- return CompletableFuture.completedFuture(Optional.of(data.sourceBroker()));
+ return getActiveOwnerAsync(serviceUnit, state, Optional.of(data.sourceBroker()));
}
case Assigning, Releasing -> {
- return deferGetOwnerRequest(serviceUnit).whenComplete((__, e) -> {
- if (e != null) {
- ownerLookUpCounters.get(state).getFailure().incrementAndGet();
- }
- }).thenApply(
- broker -> broker == null ? Optional.empty() : Optional.of(broker));
+ if (isTargetBroker(data.dstBroker())) {
+ return getActiveOwnerAsync(serviceUnit, state, Optional.of(data.dstBroker()));
+ }
+ // If this broker is not the dst broker, return the dst broker as the owner(or empty).
+ // Clients need to connect(redirect) to the dst broker anyway
+ // and wait for the dst broker to receive `Owned`.
+ // This is also required to return getOwnerAsync on the src broker immediately during unloading.
+ // Otherwise, topic creation(getOwnerAsync) could block unloading bundles,
+ // if the topic creation(getOwnerAsync) happens during unloading on the src broker.
+ return CompletableFuture.completedFuture(Optional.ofNullable(data.dstBroker()));
}
case Init, Free -> {
return CompletableFuture.completedFuture(Optional.empty());
@@ -527,6 +560,25 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) {
}
}
+ private Optional getOwner(String serviceUnit) {
+ ServiceUnitStateData data = tableview.get(serviceUnit);
+ ServiceUnitState state = state(data);
+ switch (state) {
+ case Owned -> {
+ return Optional.of(data.dstBroker());
+ }
+ case Splitting -> {
+ return Optional.of(data.sourceBroker());
+ }
+ case Init, Free -> {
+ return Optional.empty();
+ }
+ default -> {
+ return null;
+ }
+ }
+ }
+
private long getNextVersionId(String serviceUnit) {
var data = tableview.get(serviceUnit);
return getNextVersionId(data);
@@ -621,7 +673,7 @@ private void handle(String serviceUnit, ServiceUnitStateData data) {
long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet();
if (debug()) {
log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}",
- lookupServiceAddress, serviceUnit, data, totalHandledRequests);
+ brokerId, serviceUnit, data, totalHandledRequests);
}
ServiceUnitState state = state(data);
@@ -680,12 +732,12 @@ private AtomicLong getHandlerCounter(ServiceUnitStateData data, boolean total) {
private void log(Throwable e, String serviceUnit, ServiceUnitStateData data, ServiceUnitStateData next) {
if (e == null) {
- if (log.isDebugEnabled() || isTransferCommand(data)) {
+ if (debug() || isTransferCommand(data)) {
long handlerTotalCount = getHandlerTotalCounter(data).get();
long handlerFailureCount = getHandlerFailureCounter(data).get();
log.info("{} handled {} event for serviceUnit:{}, cur:{}, next:{}, "
+ "totalHandledRequests:{}, totalFailedRequests:{}",
- lookupServiceAddress, getLogEventTag(data), serviceUnit,
+ brokerId, getLogEventTag(data), serviceUnit,
data == null ? "" : data,
next == null ? "" : next,
handlerTotalCount, handlerFailureCount
@@ -696,7 +748,7 @@ lookupServiceAddress, getLogEventTag(data), serviceUnit,
long handlerFailureCount = getHandlerFailureCounter(data).incrementAndGet();
log.error("{} failed to handle {} event for serviceUnit:{}, cur:{}, next:{}, "
+ "totalHandledRequests:{}, totalFailedRequests:{}",
- lookupServiceAddress, getLogEventTag(data), serviceUnit,
+ brokerId, getLogEventTag(data), serviceUnit,
data == null ? "" : data,
next == null ? "" : next,
handlerTotalCount, handlerFailureCount,
@@ -719,6 +771,9 @@ private void handleSkippedEvent(String serviceUnit) {
private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) {
var getOwnerRequest = getOwnerRequests.remove(serviceUnit);
if (getOwnerRequest != null) {
+ if (debug()) {
+ log.info("Returned owner request for serviceUnit:{}", serviceUnit);
+ }
getOwnerRequest.complete(data.dstBroker());
}
stateChangeListeners.notify(serviceUnit, data, null);
@@ -770,9 +825,14 @@ private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) {
if (getOwnerRequest != null) {
getOwnerRequest.complete(null);
}
- stateChangeListeners.notify(serviceUnit, data, null);
+
if (isTargetBroker(data.sourceBroker())) {
- log(null, serviceUnit, data, null);
+ stateChangeListeners.notifyOnCompletion(
+ data.force() ? closeServiceUnit(serviceUnit)
+ : CompletableFuture.completedFuture(0), serviceUnit, data)
+ .whenComplete((__, e) -> log(e, serviceUnit, data, null));
+ } else {
+ stateChangeListeners.notify(serviceUnit, data, null);
}
}
@@ -822,24 +882,58 @@ private boolean isTargetBroker(String broker) {
if (broker == null) {
return false;
}
- return broker.equals(lookupServiceAddress);
+ return broker.equals(brokerId);
}
private CompletableFuture deferGetOwnerRequest(String serviceUnit) {
- return getOwnerRequests
- .computeIfAbsent(serviceUnit, k -> {
- CompletableFuture future = new CompletableFuture<>();
- future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS)
- .whenComplete((v, e) -> {
- if (e != null) {
- getOwnerRequests.remove(serviceUnit, future);
- log.warn("Failed to getOwner for serviceUnit:{}",
- serviceUnit, e);
- }
- }
- );
- return future;
+
+ var requested = new MutableObject>();
+ try {
+ return getOwnerRequests
+ .computeIfAbsent(serviceUnit, k -> {
+ var ownerBefore = getOwner(serviceUnit);
+ if (ownerBefore != null && ownerBefore.isPresent()) {
+ // Here, we do a quick active check first with the computeIfAbsent lock
+ brokerRegistry.lookupAsync(ownerBefore.get()).getNow(Optional.empty())
+ .ifPresent(__ -> requested.setValue(
+ CompletableFuture.completedFuture(ownerBefore.get())));
+
+ if (requested.getValue() != null) {
+ return requested.getValue();
+ }
+ }
+
+
+ CompletableFuture future =
+ new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis,
+ TimeUnit.MILLISECONDS)
+ .exceptionally(e -> {
+ var ownerAfter = getOwner(serviceUnit);
+ log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to "
+ + "return the current owner:{}",
+ brokerId, serviceUnit, ownerAfter, e);
+ if (ownerAfter == null) {
+ throw new IllegalStateException(e);
+ }
+ return ownerAfter.orElse(null);
+ });
+ if (debug()) {
+ log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit);
+ }
+ requested.setValue(future);
+ return future;
+ });
+ } finally {
+ var future = requested.getValue();
+ if (future != null) {
+ future.whenComplete((__, e) -> {
+ getOwnerRequests.remove(serviceUnit);
+ if (e != null) {
+ log.warn("{} failed to getOwner for serviceUnit:{}", brokerId, serviceUnit, e);
+ }
});
+ }
+ }
}
private CompletableFuture closeServiceUnit(String serviceUnit) {
@@ -1114,24 +1208,34 @@ private void handleBrokerDeletionEvent(String broker) {
}
private void scheduleCleanup(String broker, long delayInSecs) {
- cleanupJobs.computeIfAbsent(broker, k -> {
- Executor delayed = CompletableFuture
- .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor());
- totalInactiveBrokerCleanupScheduledCnt++;
- return CompletableFuture
- .runAsync(() -> {
- try {
- doCleanup(broker);
- } catch (Throwable e) {
- log.error("Failed to run the cleanup job for the broker {}, "
- + "totalCleanupErrorCnt:{}.",
- broker, totalCleanupErrorCnt.incrementAndGet(), e);
- } finally {
- cleanupJobs.remove(broker);
+ var scheduled = new MutableObject>();
+ try {
+ cleanupJobs.computeIfAbsent(broker, k -> {
+ Executor delayed = CompletableFuture
+ .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor());
+ totalInactiveBrokerCleanupScheduledCnt++;
+ var future = CompletableFuture
+ .runAsync(() -> {
+ try {
+ doCleanup(broker);
+ } catch (Throwable e) {
+ log.error("Failed to run the cleanup job for the broker {}, "
+ + "totalCleanupErrorCnt:{}.",
+ broker, totalCleanupErrorCnt.incrementAndGet(), e);
+ }
}
- }
- , delayed);
- });
+ , delayed);
+ scheduled.setValue(future);
+ return future;
+ });
+ } finally {
+ var future = scheduled.getValue();
+ if (future != null) {
+ future.whenComplete((v, ex) -> {
+ cleanupJobs.remove(broker);
+ });
+ }
+ }
log.info("Scheduled ownership cleanup for broker:{} with delay:{} secs. Pending clean jobs:{}.",
broker, delayInSecs, cleanupJobs.size());
@@ -1139,38 +1243,43 @@ private void scheduleCleanup(String broker, long delayInSecs) {
private ServiceUnitStateData getOverrideInactiveBrokerStateData(ServiceUnitStateData orphanData,
- String selectedBroker,
+ Optional selectedBroker,
String inactiveBroker) {
+
+
+ if (selectedBroker.isEmpty()) {
+ return new ServiceUnitStateData(Free, null, inactiveBroker,
+ true, getNextVersionId(orphanData));
+ }
+
if (orphanData.state() == Splitting) {
- return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker,
+ return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker.get(),
Map.copyOf(orphanData.splitServiceUnitToDestBroker()),
true, getNextVersionId(orphanData));
} else {
- return new ServiceUnitStateData(Owned, selectedBroker, inactiveBroker,
+ return new ServiceUnitStateData(Owned, selectedBroker.get(), inactiveBroker,
true, getNextVersionId(orphanData));
}
}
private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) {
Optional selectedBroker = selectBroker(serviceUnit, inactiveBroker);
- if (selectedBroker.isPresent()) {
- var override = getOverrideInactiveBrokerStateData(
- orphanData, selectedBroker.get(), inactiveBroker);
- log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}",
- serviceUnit, orphanData, override);
- publishOverrideEventAsync(serviceUnit, orphanData, override)
- .exceptionally(e -> {
- log.error(
- "Failed to override the ownership serviceUnit:{} orphanData:{}. "
- + "Failed to publish override event. totalCleanupErrorCnt:{}",
- serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet());
- return null;
- });
- } else {
- log.error("Failed to override the ownership serviceUnit:{} orphanData:{}. Empty selected broker. "
+ if (selectedBroker.isEmpty()) {
+ log.warn("Empty selected broker for ownership serviceUnit:{} orphanData:{}."
+ "totalCleanupErrorCnt:{}",
serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet());
}
+ var override = getOverrideInactiveBrokerStateData(orphanData, selectedBroker, inactiveBroker);
+ log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}",
+ serviceUnit, orphanData, override);
+ publishOverrideEventAsync(serviceUnit, orphanData, override)
+ .exceptionally(e -> {
+ log.error(
+ "Failed to override the ownership serviceUnit:{} orphanData:{}. "
+ + "Failed to publish override event. totalCleanupErrorCnt:{}",
+ serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet());
+ return null;
+ });
}
private void waitForCleanups(String broker, boolean excludeSystemTopics, int maxWaitTimeInMillis) {
@@ -1202,7 +1311,7 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max
MILLISECONDS.sleep(OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS);
} catch (InterruptedException e) {
log.warn("Interrupted while delaying the next service unit clean-up. Cleaning broker:{}",
- lookupServiceAddress);
+ brokerId);
}
}
}
@@ -1213,49 +1322,19 @@ private synchronized void doCleanup(String broker) {
log.info("Started ownership cleanup for the inactive broker:{}", broker);
int orphanServiceUnitCleanupCnt = 0;
long totalCleanupErrorCntStart = totalCleanupErrorCnt.get();
- String heartbeatNamespace =
- NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration())
- .toString();
- String heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(),
- pulsar.getConfiguration()).toString();
-
Map orphanSystemServiceUnits = new HashMap<>();
for (var etr : tableview.entrySet()) {
var stateData = etr.getValue();
var serviceUnit = etr.getKey();
var state = state(stateData);
- if (StringUtils.equals(broker, stateData.dstBroker())) {
- if (isActiveState(state)) {
- if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) {
- orphanSystemServiceUnits.put(serviceUnit, stateData);
- } else if (serviceUnit.startsWith(heartbeatNamespace)
- || serviceUnit.startsWith(heartbeatNamespaceV2)) {
- // Skip the heartbeat namespace
- log.info("Skip override heartbeat namespace bundle"
- + " serviceUnit:{}, stateData:{}", serviceUnit, stateData);
- tombstoneAsync(serviceUnit).whenComplete((__, e) -> {
- if (e != null) {
- log.error("Failed cleaning the heartbeat namespace ownership serviceUnit:{}, "
- + "stateData:{}, cleanupErrorCnt:{}.",
- serviceUnit, stateData,
- totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e);
- }
- });
- } else {
- overrideOwnership(serviceUnit, stateData, broker);
- }
- orphanServiceUnitCleanupCnt++;
- }
-
- } else if (StringUtils.equals(broker, stateData.sourceBroker())) {
- if (isInFlightState(state)) {
- if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) {
- orphanSystemServiceUnits.put(serviceUnit, stateData);
- } else {
- overrideOwnership(serviceUnit, stateData, broker);
- }
- orphanServiceUnitCleanupCnt++;
+ if (StringUtils.equals(broker, stateData.dstBroker()) && isActiveState(state)
+ || StringUtils.equals(broker, stateData.sourceBroker()) && isInFlightState(state)) {
+ if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) {
+ orphanSystemServiceUnits.put(serviceUnit, stateData);
+ } else {
+ overrideOwnership(serviceUnit, stateData, broker);
}
+ orphanServiceUnitCleanupCnt++;
}
}
@@ -1302,7 +1381,7 @@ private synchronized void doCleanup(String broker) {
broker,
cleanupTime,
orphanServiceUnitCleanupCnt,
- totalCleanupErrorCntStart - totalCleanupErrorCnt.get(),
+ totalCleanupErrorCnt.get() - totalCleanupErrorCntStart,
printCleanupMetrics());
}
@@ -1399,16 +1478,21 @@ protected void monitorOwnerships(List brokers) {
String srcBroker = stateData.sourceBroker();
var state = stateData.state();
- if (isActiveState(state)) {
- if (StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) {
- inactiveBrokers.add(srcBroker);
- } else if (StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) {
- inactiveBrokers.add(dstBroker);
- } else if (isInFlightState(state)
- && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) {
- orphanServiceUnits.put(serviceUnit, stateData);
- }
- } else if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) {
+ if (isActiveState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) {
+ inactiveBrokers.add(srcBroker);
+ continue;
+ }
+ if (isActiveState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) {
+ inactiveBrokers.add(dstBroker);
+ continue;
+ }
+ if (isActiveState(state) && isInFlightState(state)
+ && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) {
+ orphanServiceUnits.put(serviceUnit, stateData);
+ continue;
+ }
+
+ if (!isActiveState(state) && now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) {
log.info("Found semi-terminal states to tombstone"
+ " serviceUnit:{}, stateData:{}", serviceUnit, stateData);
tombstoneAsync(serviceUnit).whenComplete((__, e) -> {
@@ -1486,7 +1570,7 @@ protected void monitorOwnerships(List brokers) {
inactiveBrokers, inactiveBrokers.size(),
orphanServiceUnitCleanupCnt,
serviceUnitTombstoneCleanupCnt,
- totalCleanupErrorCntStart - totalCleanupErrorCnt.get(),
+ totalCleanupErrorCnt.get() - totalCleanupErrorCntStart,
printCleanupMetrics());
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java
index 2dde0c4708e41..ffdbbc2af4219 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java
@@ -88,6 +88,13 @@ public CompletableFuture waitAsync(CompletableFuture eventPubFuture,
@Override
public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) {
+ if (t != null && inFlightUnloadRequest.containsKey(serviceUnit)) {
+ if (log.isDebugEnabled()) {
+ log.debug("Handling {} for service unit {} with exception.", data, serviceUnit, t);
+ }
+ this.complete(serviceUnit, t);
+ return;
+ }
ServiceUnitState state = ServiceUnitStateData.state(data);
switch (state) {
case Free, Owned -> this.complete(serviceUnit, t);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java
index 2f5c32197c1fd..624546fdff837 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java
@@ -30,6 +30,8 @@
import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData;
import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared;
import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies;
+import org.apache.pulsar.broker.namespace.NamespaceService;
+import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats;
@@ -70,7 +72,8 @@ public void update(Map bundleStats, int topk) {
pulsar.getConfiguration().isLoadBalancerSheddingBundlesWithPoliciesEnabled();
for (var etr : bundleStats.entrySet()) {
String bundle = etr.getKey();
- if (bundle.startsWith(NamespaceName.SYSTEM_NAMESPACE.toString())) {
+ // TODO: do not filter system topic while shedding
+ if (NamespaceService.isSystemServiceNamespace(NamespaceBundle.getBundleNamespace(bundle))) {
continue;
}
if (!isLoadBalancerSheddingBundlesWithPoliciesEnabled && hasPolicies(bundle)) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/policies/IsolationPoliciesHelper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/policies/IsolationPoliciesHelper.java
index 468552db541ec..56238d6528e60 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/policies/IsolationPoliciesHelper.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/policies/IsolationPoliciesHelper.java
@@ -42,14 +42,14 @@ public CompletableFuture> applyIsolationPoliciesAsync(Map,
private final BrokerHostUsage brokerHostUsage;
- private final String lookupServiceAddress;
+ private final String brokerId;
@Getter
private final BrokerLoadData localData;
@@ -67,10 +67,10 @@ public class BrokerLoadDataReporter implements LoadDataReporter,
private long tombstoneDelayInMillis;
public BrokerLoadDataReporter(PulsarService pulsar,
- String lookupServiceAddress,
+ String brokerId,
LoadDataStore brokerLoadDataStore) {
this.brokerLoadDataStore = brokerLoadDataStore;
- this.lookupServiceAddress = lookupServiceAddress;
+ this.brokerId = brokerId;
this.pulsar = pulsar;
this.conf = this.pulsar.getConfiguration();
if (SystemUtils.IS_OS_LINUX) {
@@ -111,7 +111,7 @@ public CompletableFuture reportAsync(boolean force) {
log.info("publishing load report:{}", localData.toString(conf));
}
CompletableFuture future =
- this.brokerLoadDataStore.pushAsync(this.lookupServiceAddress, newLoadData);
+ this.brokerLoadDataStore.pushAsync(this.brokerId, newLoadData);
future.whenComplete((__, ex) -> {
if (ex == null) {
localData.setReportedAt(System.currentTimeMillis());
@@ -185,7 +185,7 @@ protected void tombstone() {
}
var lastSuccessfulTombstonedAt = lastTombstonedAt;
lastTombstonedAt = now; // dedup first
- brokerLoadDataStore.removeAsync(lookupServiceAddress)
+ brokerLoadDataStore.removeAsync(brokerId)
.whenComplete((__, e) -> {
if (e != null) {
log.error("Failed to clean broker load data.", e);
@@ -209,13 +209,13 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable
ServiceUnitState state = ServiceUnitStateData.state(data);
switch (state) {
case Releasing, Splitting -> {
- if (StringUtils.equals(data.sourceBroker(), lookupServiceAddress)) {
+ if (StringUtils.equals(data.sourceBroker(), brokerId)) {
localData.clear();
tombstone();
}
}
case Owned -> {
- if (StringUtils.equals(data.dstBroker(), lookupServiceAddress)) {
+ if (StringUtils.equals(data.dstBroker(), brokerId)) {
localData.clear();
tombstone();
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/reporter/TopBundleLoadDataReporter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/reporter/TopBundleLoadDataReporter.java
index 0fa37d3687c20..43e05ad1ac972 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/reporter/TopBundleLoadDataReporter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/reporter/TopBundleLoadDataReporter.java
@@ -41,7 +41,7 @@ public class TopBundleLoadDataReporter implements LoadDataReporter bundleLoadDataStore;
@@ -53,10 +53,10 @@ public class TopBundleLoadDataReporter implements LoadDataReporter bundleLoadDataStore) {
this.pulsar = pulsar;
- this.lookupServiceAddress = lookupServiceAddress;
+ this.brokerId = brokerId;
this.bundleLoadDataStore = bundleLoadDataStore;
this.lastBundleStatsUpdatedAt = 0;
this.topKBundles = new TopKBundles(pulsar);
@@ -88,7 +88,7 @@ public CompletableFuture reportAsync(boolean force) {
if (ExtensibleLoadManagerImpl.debug(pulsar.getConfiguration(), log)) {
log.info("Reporting TopBundlesLoadData:{}", topKBundles.getLoadData());
}
- return this.bundleLoadDataStore.pushAsync(lookupServiceAddress, topKBundles.getLoadData())
+ return this.bundleLoadDataStore.pushAsync(brokerId, topKBundles.getLoadData())
.exceptionally(e -> {
log.error("Failed to report top-bundles load data.", e);
return null;
@@ -106,7 +106,7 @@ protected void tombstone() {
}
var lastSuccessfulTombstonedAt = lastTombstonedAt;
lastTombstonedAt = now; // dedup first
- bundleLoadDataStore.removeAsync(lookupServiceAddress)
+ bundleLoadDataStore.removeAsync(brokerId)
.whenComplete((__, e) -> {
if (e != null) {
log.error("Failed to clean broker load data.", e);
@@ -129,12 +129,12 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable
ServiceUnitState state = ServiceUnitStateData.state(data);
switch (state) {
case Releasing, Splitting -> {
- if (StringUtils.equals(data.sourceBroker(), lookupServiceAddress)) {
+ if (StringUtils.equals(data.sourceBroker(), brokerId)) {
tombstone();
}
}
case Owned -> {
- if (StringUtils.equals(data.dstBroker(), lookupServiceAddress)) {
+ if (StringUtils.equals(data.dstBroker(), brokerId)) {
tombstone();
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/UnloadScheduler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/UnloadScheduler.java
index d6c754c90fcf6..218f57932a56b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/UnloadScheduler.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/UnloadScheduler.java
@@ -219,9 +219,8 @@ private static NamespaceUnloadStrategy createNamespaceUnloadStrategy(PulsarServi
Thread.currentThread().getContextClassLoader());
log.info("Created namespace unload strategy:{}", unloadStrategy.getClass().getCanonicalName());
} catch (Exception e) {
- log.error("Error when trying to create namespace unload strategy: {}",
- conf.getLoadBalancerLoadPlacementStrategy(), e);
- log.error("create namespace unload strategy failed. using TransferShedder instead.");
+ log.error("Error when trying to create namespace unload strategy: {}. Using {} instead.",
+ conf.getLoadBalancerLoadSheddingStrategy(), TransferShedder.class.getCanonicalName(), e);
unloadStrategy = new TransferShedder();
}
unloadStrategy.initialize(pulsar);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java
index 680a36523a214..a7deeeaad8a5c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java
@@ -81,9 +81,26 @@ public interface LoadDataStore extends Closeable {
*/
void closeTableView() throws IOException;
+
+ /**
+ * Starts the data store (both producer and table view).
+ */
+ void start() throws LoadDataStoreException;
+
+ /**
+ * Inits the data store (close and start the data store).
+ */
+ void init() throws IOException;
+
/**
* Starts the table view.
*/
void startTableView() throws LoadDataStoreException;
+
+ /**
+ * Starts the producer.
+ */
+ void startProducer() throws LoadDataStoreException;
+
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java
index 18f39abd76b76..bcb2657c67f05 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java
@@ -18,15 +18,16 @@
*/
package org.apache.pulsar.broker.loadbalance.extensions.store;
-import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.broker.PulsarService;
/**
* The load data store factory, use to create the load data store.
*/
public class LoadDataStoreFactory {
- public static LoadDataStore create(PulsarClient client, String name, Class clazz)
+ public static LoadDataStore create(PulsarService pulsar, String name,
+ Class clazz)
throws LoadDataStoreException {
- return new TableViewLoadDataStoreImpl<>(client, name, clazz);
+ return new TableViewLoadDataStoreImpl<>(pulsar, name, clazz);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java
index a400163ebf122..d916e91716223 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java
@@ -23,7 +23,12 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
@@ -35,11 +40,17 @@
*
* @param Load data type.
*/
+@Slf4j
public class TableViewLoadDataStoreImpl implements LoadDataStore {
- private TableView tableView;
+ private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2;
- private final Producer producer;
+ private volatile TableView tableView;
+ private volatile long tableViewLastUpdateTimestamp;
+
+ private volatile Producer producer;
+
+ private final ServiceConfiguration conf;
private final PulsarClient client;
@@ -47,10 +58,11 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore {
private final Class clazz;
- public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class clazz) throws LoadDataStoreException {
+ public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class clazz)
+ throws LoadDataStoreException {
try {
- this.client = client;
- this.producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create();
+ this.conf = pulsar.getConfiguration();
+ this.client = pulsar.getClient();
this.topic = topic;
this.clazz = clazz;
} catch (Exception e) {
@@ -59,40 +71,42 @@ public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class cl
}
@Override
- public CompletableFuture pushAsync(String key, T loadData) {
+ public synchronized CompletableFuture pushAsync(String key, T loadData) {
+ validateProducer();
return producer.newMessage().key(key).value(loadData).sendAsync().thenAccept(__ -> {});
}
@Override
- public CompletableFuture removeAsync(String key) {
+ public synchronized CompletableFuture removeAsync(String key) {
+ validateProducer();
return producer.newMessage().key(key).value(null).sendAsync().thenAccept(__ -> {});
}
@Override
- public Optional get(String key) {
- validateTableViewStart();
+ public synchronized Optional get(String key) {
+ validateTableView();
return Optional.ofNullable(tableView.get(key));
}
@Override
- public void forEach(BiConsumer action) {
- validateTableViewStart();
+ public synchronized void forEach(BiConsumer action) {
+ validateTableView();
tableView.forEach(action);
}
- public Set> entrySet() {
- validateTableViewStart();
+ public synchronized Set> entrySet() {
+ validateTableView();
return tableView.entrySet();
}
@Override
- public int size() {
- validateTableViewStart();
+ public synchronized int size() {
+ validateTableView();
return tableView.size();
}
@Override
- public void closeTableView() throws IOException {
+ public synchronized void closeTableView() throws IOException {
if (tableView != null) {
tableView.close();
tableView = null;
@@ -100,10 +114,18 @@ public void closeTableView() throws IOException {
}
@Override
- public void startTableView() throws LoadDataStoreException {
+ public synchronized void start() throws LoadDataStoreException {
+ startProducer();
+ startTableView();
+ }
+
+ @Override
+ public synchronized void startTableView() throws LoadDataStoreException {
if (tableView == null) {
try {
tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).create();
+ tableView.forEachAndListen((k, v) ->
+ tableViewLastUpdateTimestamp = System.currentTimeMillis());
} catch (PulsarClientException e) {
tableView = null;
throw new LoadDataStoreException(e);
@@ -112,17 +134,74 @@ public void startTableView() throws LoadDataStoreException {
}
@Override
- public void close() throws IOException {
+ public synchronized void startProducer() throws LoadDataStoreException {
+ if (producer == null) {
+ try {
+ producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create();
+ } catch (PulsarClientException e) {
+ producer = null;
+ throw new LoadDataStoreException(e);
+ }
+ }
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
if (producer != null) {
producer.close();
+ producer = null;
}
closeTableView();
}
- private void validateTableViewStart() {
- if (tableView == null) {
- throw new IllegalStateException("table view has not been started");
+ @Override
+ public synchronized void init() throws IOException {
+ close();
+ start();
+ }
+
+ private void validateProducer() {
+ if (producer == null || !producer.isConnected()) {
+ try {
+ if (producer != null) {
+ producer.close();
+ }
+ producer = null;
+ startProducer();
+ log.info("Restarted producer on {}", topic);
+ } catch (Exception e) {
+ log.error("Failed to restart producer on {}", topic, e);
+ throw new RuntimeException(e);
+ }
}
}
+ private void validateTableView() {
+ String restartReason = null;
+
+ if (tableView == null) {
+ restartReason = "table view is null";
+ } else {
+ long inactiveDuration = System.currentTimeMillis() - tableViewLastUpdateTimestamp;
+ long threshold = TimeUnit.MINUTES.toMillis(conf.getLoadBalancerReportUpdateMaxIntervalMinutes())
+ * LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART;
+ if (inactiveDuration > threshold) {
+ restartReason = String.format("inactiveDuration=%d secs > threshold = %d secs",
+ TimeUnit.MILLISECONDS.toSeconds(inactiveDuration),
+ TimeUnit.MILLISECONDS.toSeconds(threshold));
+ }
+ }
+
+ if (StringUtils.isNotBlank(restartReason)) {
+ tableViewLastUpdateTimestamp = 0;
+ try {
+ closeTableView();
+ startTableView();
+ log.info("Restarted tableview on {}, {}", topic, restartReason);
+ } catch (Exception e) {
+ log.error("Failed to restart tableview on {}", topic, e);
+ throw new RuntimeException(e);
+ }
+ }
+ }
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java
index 2f7ca614943b1..6d0e6bb907346 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java
@@ -155,7 +155,8 @@ private double getTotalCpuUsageForCGroup(double elapsedTimeSeconds) {
*
*
* Line is split in "words", filtering the first. The sum of all numbers give the amount of cpu cycles used this
- * far. Real CPU usage should equal the sum subtracting the idle cycles, this would include iowait, irq and steal.
+ * far. Real CPU usage should equal the sum substracting the idle cycles(that is idle+iowait), this would include
+ * cpu, user, nice, system, irq, softirq, steal, guest and guest_nice.
*/
private double getTotalCpuUsageForEntireHost() {
LinuxInfoUtils.ResourceUsage cpuUsageForEntireHost = getCpuUsageForEntireHost();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
index 5f2e4b1f25d8b..3d627db6cfa9e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
@@ -21,8 +21,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed;
import io.netty.util.concurrent.FastThreadLocal;
-import java.net.MalformedURLException;
-import java.net.URL;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
@@ -106,59 +104,56 @@ public static void applyNamespacePolicies(final ServiceUnitId serviceUnit,
if (isIsolationPoliciesPresent) {
LOG.debug("Isolation Policies Present for namespace - [{}]", namespace.toString());
}
- for (final String broker : availableBrokers) {
- final String brokerUrlString = String.format("http://%s", broker);
- URL brokerUrl;
+ for (final String brokerId : availableBrokers) {
+ String brokerHost;
try {
- brokerUrl = new URL(brokerUrlString);
- } catch (MalformedURLException e) {
- LOG.error("Unable to parse brokerUrl from ResourceUnitId", e);
+ brokerHost = parseBrokerHost(brokerId);
+ } catch (IllegalArgumentException e) {
+ LOG.error("Unable to parse host from {}", brokerId, e);
continue;
}
// todo: in future check if the resource unit has resources to take the namespace
if (isIsolationPoliciesPresent) {
// note: serviceUnitID is namespace name and ResourceID is brokerName
- if (policies.isPrimaryBroker(namespace, brokerUrl.getHost())) {
- primariesCache.add(broker);
+ if (policies.isPrimaryBroker(namespace, brokerHost)) {
+ primariesCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Added Primary Broker - [{}] as possible Candidates for"
- + " namespace - [{}] with policies", brokerUrl.getHost(), namespace.toString());
+ + " namespace - [{}] with policies", brokerHost, namespace.toString());
}
- } else if (policies.isSecondaryBroker(namespace, brokerUrl.getHost())) {
- secondaryCache.add(broker);
+ } else if (policies.isSecondaryBroker(namespace, brokerHost)) {
+ secondaryCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug(
"Added Shared Broker - [{}] as possible "
+ "Candidates for namespace - [{}] with policies",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping Broker - [{}] not primary broker and not shared" + " for namespace - [{}] ",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
}
} else {
// non-persistent topic can be assigned to only those brokers that enabled for non-persistent topic
- if (isNonPersistentTopic
- && !brokerTopicLoadingPredicate.isEnableNonPersistentTopics(brokerUrlString)) {
+ if (isNonPersistentTopic && !brokerTopicLoadingPredicate.isEnableNonPersistentTopics(brokerId)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Filter broker- [{}] because it doesn't support non-persistent namespace - [{}]",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
- } else if (!isNonPersistentTopic
- && !brokerTopicLoadingPredicate.isEnablePersistentTopics(brokerUrlString)) {
+ } else if (!isNonPersistentTopic && !brokerTopicLoadingPredicate.isEnablePersistentTopics(brokerId)) {
// persistent topic can be assigned to only brokers that enabled for persistent-topic
if (LOG.isDebugEnabled()) {
LOG.debug("Filter broker- [{}] because broker only supports non-persistent namespace - [{}]",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
- } else if (policies.isSharedBroker(brokerUrl.getHost())) {
- secondaryCache.add(broker);
+ } else if (policies.isSharedBroker(brokerHost)) {
+ secondaryCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Added Shared Broker - [{}] as possible Candidates for namespace - [{}]",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
}
}
@@ -181,6 +176,16 @@ public static void applyNamespacePolicies(final ServiceUnitId serviceUnit,
}
}
+ private static String parseBrokerHost(String brokerId) {
+ // use last index to support ipv6 addresses
+ int lastIdx = brokerId.lastIndexOf(':');
+ if (lastIdx > -1) {
+ return brokerId.substring(0, lastIdx);
+ } else {
+ throw new IllegalArgumentException("Invalid brokerId: " + brokerId);
+ }
+ }
+
public static CompletableFuture> applyNamespacePoliciesAsync(
final ServiceUnitId serviceUnit, final SimpleResourceAllocationPolicies policies,
final Set availableBrokers, final BrokerTopicLoadingPredicate brokerTopicLoadingPredicate) {
@@ -199,59 +204,57 @@ public static CompletableFuture> applyNamespacePoliciesAsync(
LOG.debug("Isolation Policies Present for namespace - [{}]", namespace.toString());
}
}
- for (final String broker : availableBrokers) {
- final String brokerUrlString = String.format("http://%s", broker);
- URL brokerUrl;
+ for (final String brokerId : availableBrokers) {
+ String brokerHost;
try {
- brokerUrl = new URL(brokerUrlString);
- } catch (MalformedURLException e) {
- LOG.error("Unable to parse brokerUrl from ResourceUnitId", e);
+ brokerHost = parseBrokerHost(brokerId);
+ } catch (IllegalArgumentException e) {
+ LOG.error("Unable to parse host from {}", brokerId, e);
continue;
}
// todo: in future check if the resource unit has resources to take the namespace
if (isIsolationPoliciesPresent) {
// note: serviceUnitID is namespace name and ResourceID is brokerName
- if (policies.isPrimaryBroker(namespace, brokerUrl.getHost())) {
- primariesCache.add(broker);
+ if (policies.isPrimaryBroker(namespace, brokerHost)) {
+ primariesCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Added Primary Broker - [{}] as possible Candidates for"
- + " namespace - [{}] with policies", brokerUrl.getHost(), namespace.toString());
+ + " namespace - [{}] with policies", brokerHost, namespace.toString());
}
- } else if (policies.isSecondaryBroker(namespace, brokerUrl.getHost())) {
- secondaryCache.add(broker);
+ } else if (policies.isSecondaryBroker(namespace, brokerHost)) {
+ secondaryCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug(
"Added Shared Broker - [{}] as possible "
+ "Candidates for namespace - [{}] with policies",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping Broker - [{}] not primary broker and not shared"
- + " for namespace - [{}] ", brokerUrl.getHost(), namespace.toString());
+ + " for namespace - [{}] ", brokerHost, namespace.toString());
}
}
} else {
// non-persistent topic can be assigned to only those brokers that enabled for non-persistent topic
- if (isNonPersistentTopic
- && !brokerTopicLoadingPredicate.isEnableNonPersistentTopics(brokerUrlString)) {
+ if (isNonPersistentTopic && !brokerTopicLoadingPredicate.isEnableNonPersistentTopics(brokerId)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Filter broker- [{}] because it doesn't support non-persistent namespace - [{}]",
- brokerUrl.getHost(), namespace.toString());
+ brokerId, namespace.toString());
}
- } else if (!isNonPersistentTopic
- && !brokerTopicLoadingPredicate.isEnablePersistentTopics(brokerUrlString)) {
+ } else if (!isNonPersistentTopic && !brokerTopicLoadingPredicate
+ .isEnablePersistentTopics(brokerId)) {
// persistent topic can be assigned to only brokers that enabled for persistent-topic
if (LOG.isDebugEnabled()) {
LOG.debug("Filter broker- [{}] because broker only supports non-persistent "
- + "namespace - [{}]", brokerUrl.getHost(), namespace.toString());
+ + "namespace - [{}]", brokerId, namespace.toString());
}
- } else if (policies.isSharedBroker(brokerUrl.getHost())) {
- secondaryCache.add(broker);
+ } else if (policies.isSharedBroker(brokerHost)) {
+ secondaryCache.add(brokerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Added Shared Broker - [{}] as possible Candidates for namespace - [{}]",
- brokerUrl.getHost(), namespace.toString());
+ brokerHost, namespace.toString());
}
}
}
@@ -762,9 +765,9 @@ public static boolean shouldAntiAffinityNamespaceUnload(
}
public interface BrokerTopicLoadingPredicate {
- boolean isEnablePersistentTopics(String brokerUrl);
+ boolean isEnablePersistentTopics(String brokerId);
- boolean isEnableNonPersistentTopics(String brokerUrl);
+ boolean isEnableNonPersistentTopics(String brokerId);
}
/**
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
index 8882c2cc6c56b..320c273a2d9b7 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
@@ -108,7 +108,7 @@ public class ModularLoadManagerImpl implements ModularLoadManager {
public static final int NUM_SHORT_SAMPLES = 10;
// Path to ZNode whose children contain ResourceQuota jsons.
- public static final String RESOURCE_QUOTA_ZPATH = "/loadbalance/resource-quota/namespace";
+ public static final String RESOURCE_QUOTA_ZPATH = "/loadbalance/resource-quota";
// Path to ZNode containing TimeAverageBrokerData jsons for each broker.
public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average";
@@ -219,15 +219,15 @@ public ModularLoadManagerImpl() {
this.bundleBrokerAffinityMap = new ConcurrentHashMap<>();
this.brokerTopicLoadingPredicate = new BrokerTopicLoadingPredicate() {
@Override
- public boolean isEnablePersistentTopics(String brokerUrl) {
- final BrokerData brokerData = loadData.getBrokerData().get(brokerUrl.replace("http://", ""));
+ public boolean isEnablePersistentTopics(String brokerId) {
+ final BrokerData brokerData = loadData.getBrokerData().get(brokerId);
return brokerData != null && brokerData.getLocalData() != null
&& brokerData.getLocalData().isPersistentTopicsEnabled();
}
@Override
- public boolean isEnableNonPersistentTopics(String brokerUrl) {
- final BrokerData brokerData = loadData.getBrokerData().get(brokerUrl.replace("http://", ""));
+ public boolean isEnableNonPersistentTopics(String brokerId) {
+ final BrokerData brokerData = loadData.getBrokerData().get(brokerId);
return brokerData != null && brokerData.getLocalData() != null
&& brokerData.getLocalData().isNonPersistentTopicsEnabled();
}
@@ -559,17 +559,6 @@ private void updateBundleData() {
bundleData.put(bundle, currentBundleData);
}
}
-
- //Remove not active bundle from loadData
- for (String bundle : bundleData.keySet()) {
- if (!activeBundles.contains(bundle)){
- bundleData.remove(bundle);
- if (pulsar.getLeaderElectionService().isLeader()){
- deleteBundleDataFromMetadataStore(bundle);
- }
- }
- }
-
// Remove all loaded bundles from the preallocated maps.
final Map preallocatedBundleData = brokerData.getPreallocatedBundleData();
Set ownedNsBundles = pulsar.getNamespaceService().getOwnedServiceUnits()
@@ -604,6 +593,16 @@ private void updateBundleData() {
LoadManagerShared.fillNamespaceToBundlesMap(preallocatedBundleData.keySet(), namespaceToBundleRange);
}
}
+
+ // Remove not active bundle from loadData
+ for (String bundle : bundleData.keySet()) {
+ if (!activeBundles.contains(bundle)){
+ bundleData.remove(bundle);
+ if (pulsar.getLeaderElectionService().isLeader()){
+ deleteBundleDataFromMetadataStore(bundle);
+ }
+ }
+ }
}
/**
@@ -979,14 +978,14 @@ public void start() throws PulsarServerException {
// At this point, the ports will be updated with the real port number that the server was assigned
Map protocolData = pulsar.getProtocolDataToAdvertise();
- lastData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(),
+ lastData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners());
lastData.setProtocols(protocolData);
// configure broker-topic mode
lastData.setPersistentTopicsEnabled(pulsar.getConfiguration().isEnablePersistentTopics());
lastData.setNonPersistentTopicsEnabled(pulsar.getConfiguration().isEnableNonPersistentTopics());
- localData = new LocalBrokerData(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(),
+ localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getAdvertisedListeners());
localData.setProtocols(protocolData);
localData.setBrokerVersionString(pulsar.getBrokerVersion());
@@ -995,9 +994,9 @@ public void start() throws PulsarServerException {
localData.setNonPersistentTopicsEnabled(pulsar.getConfiguration().isEnableNonPersistentTopics());
localData.setLoadManagerClassName(conf.getLoadManagerClassName());
- String lookupServiceAddress = pulsar.getLookupServiceAddress();
- brokerZnodePath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress;
- final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + lookupServiceAddress;
+ String brokerId = pulsar.getBrokerId();
+ brokerZnodePath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + brokerId;
+ final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + brokerId;
updateLocalBrokerData();
brokerDataLock = brokersData.acquireLock(brokerZnodePath, localData).join();
@@ -1231,7 +1230,6 @@ public String setNamespaceBundleAffinity(String bundle, String broker) {
if (StringUtils.isBlank(broker)) {
return this.bundleBrokerAffinityMap.remove(bundle);
}
- broker = broker.replaceFirst("http[s]?://", "");
return this.bundleBrokerAffinityMap.put(bundle, broker);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java
index c61d39cf3159a..c8d81bda1bc13 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java
@@ -19,7 +19,6 @@
package org.apache.pulsar.broker.loadbalance.impl;
import java.util.List;
-import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
@@ -32,7 +31,6 @@
import org.apache.pulsar.common.naming.ServiceUnitId;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
-import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData;
/**
* Wrapper class allowing classes of instance ModularLoadManager to be compatible with the interface LoadManager.
@@ -75,20 +73,6 @@ public Optional getLeastLoaded(final ServiceUnitId serviceUnit) {
return leastLoadedBroker.map(this::buildBrokerResourceUnit);
}
- private String getBrokerWebServiceUrl(String broker) {
- LocalBrokerData localData = (loadManager).getBrokerLocalData(broker);
- if (localData != null) {
- return localData.getWebServiceUrl() != null ? localData.getWebServiceUrl()
- : localData.getWebServiceUrlTls();
- }
- return String.format("http://%s", broker);
- }
-
- private String getBrokerZnodeName(String broker, String webServiceUrl) {
- String scheme = webServiceUrl.substring(0, webServiceUrl.indexOf("://"));
- return String.format("%s://%s", scheme, broker);
- }
-
@Override
public List getLoadBalancingMetrics() {
return loadManager.getLoadBalancingMetrics();
@@ -149,10 +133,7 @@ public CompletableFuture> getAvailableBrokersAsync() {
}
private SimpleResourceUnit buildBrokerResourceUnit (String broker) {
- String webServiceUrl = getBrokerWebServiceUrl(broker);
- String brokerZnodeName = getBrokerZnodeName(broker, webServiceUrl);
- return new SimpleResourceUnit(webServiceUrl,
- new PulsarResourceDescription(), Map.of(ResourceUnit.PROPERTY_KEY_BROKER_ZNODE_NAME, brokerZnodeName));
+ return new SimpleResourceUnit(broker, new PulsarResourceDescription());
}
@Override
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
index 5e99456971147..8bb27e52298be 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
@@ -211,15 +211,15 @@ public SimpleLoadManagerImpl() {
.build();
this.brokerTopicLoadingPredicate = new BrokerTopicLoadingPredicate() {
@Override
- public boolean isEnablePersistentTopics(String brokerUrl) {
- ResourceUnit ru = new SimpleResourceUnit(brokerUrl, new PulsarResourceDescription());
+ public boolean isEnablePersistentTopics(String brokerId) {
+ ResourceUnit ru = new SimpleResourceUnit(brokerId, new PulsarResourceDescription());
LoadReport loadReport = currentLoadReports.get(ru);
return loadReport != null && loadReport.isPersistentTopicsEnabled();
}
@Override
- public boolean isEnableNonPersistentTopics(String brokerUrl) {
- ResourceUnit ru = new SimpleResourceUnit(brokerUrl, new PulsarResourceDescription());
+ public boolean isEnableNonPersistentTopics(String brokerId) {
+ ResourceUnit ru = new SimpleResourceUnit(brokerId, new PulsarResourceDescription());
LoadReport loadReport = currentLoadReports.get(ru);
return loadReport != null && loadReport.isNonPersistentTopicsEnabled();
}
@@ -234,7 +234,7 @@ public void initialize(final PulsarService pulsar) {
brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar);
}
this.policies = new SimpleResourceAllocationPolicies(pulsar);
- lastLoadReport = new LoadReport(pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(),
+ lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls());
lastLoadReport.setProtocols(pulsar.getProtocolDataToAdvertise());
lastLoadReport.setPersistentTopicsEnabled(pulsar.getConfiguration().isEnablePersistentTopics());
@@ -266,8 +266,8 @@ public SimpleLoadManagerImpl(PulsarService pulsar) {
@Override
public void start() throws PulsarServerException {
// Register the brokers in metadata store
- String lookupServiceAddress = pulsar.getLookupServiceAddress();
- String brokerLockPath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress;
+ String brokerId = pulsar.getBrokerId();
+ String brokerLockPath = LOADBALANCE_BROKERS_ROOT + "/" + brokerId;
try {
LoadReport loadReport = null;
@@ -653,7 +653,6 @@ public void writeResourceQuotasToZooKeeper() throws Exception {
*/
private synchronized void doLoadRanking() {
ResourceUnitRanking.setCpuUsageByMsgRate(this.realtimeCpuLoadFactor);
- String hostname = pulsar.getAdvertisedAddress();
String strategy = this.getLoadBalancerPlacementStrategy();
log.info("doLoadRanking - load balancing strategy: {}", strategy);
if (!currentLoadReports.isEmpty()) {
@@ -702,8 +701,8 @@ private synchronized void doLoadRanking() {
}
// update metrics
- if (resourceUnit.getResourceId().contains(hostname)) {
- updateLoadBalancingMetrics(hostname, finalRank, ranking);
+ if (resourceUnit.getResourceId().equals(pulsar.getBrokerId())) {
+ updateLoadBalancingMetrics(pulsar.getAdvertisedAddress(), finalRank, ranking);
}
}
updateBrokerToNamespaceToBundle();
@@ -711,7 +710,7 @@ private synchronized void doLoadRanking() {
this.resourceUnitRankings = newResourceUnitRankings;
} else {
log.info("Leader broker[{}] No ResourceUnits to rank this run, Using Old Ranking",
- pulsar.getSafeWebServiceAddress());
+ pulsar.getBrokerId());
}
}
@@ -855,7 +854,7 @@ private synchronized ResourceUnit findBrokerForPlacement(Multimap ConcurrentOpenHashMap.>newBuilder()
.build())
@@ -876,7 +875,7 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni
availableBrokersCache.clear();
for (final Set resourceUnits : availableBrokers.values()) {
for (final ResourceUnit resourceUnit : resourceUnits) {
- availableBrokersCache.add(resourceUnit.getResourceId().replace("http://", ""));
+ availableBrokersCache.add(resourceUnit.getResourceId());
}
}
brokerCandidateCache.clear();
@@ -899,7 +898,7 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni
final Long rank = entry.getKey();
final Set resourceUnits = entry.getValue();
for (final ResourceUnit resourceUnit : resourceUnits) {
- if (brokerCandidateCache.contains(resourceUnit.getResourceId().replace("http://", ""))) {
+ if (brokerCandidateCache.contains(resourceUnit.getResourceId())) {
result.put(rank, resourceUnit);
}
}
@@ -928,8 +927,7 @@ private Map> getAvailableBrokers(ServiceUnitId serviceUn
availableBrokers = new HashMap<>();
for (String broker : activeBrokers) {
- ResourceUnit resourceUnit = new SimpleResourceUnit(String.format("http://%s", broker),
- new PulsarResourceDescription());
+ ResourceUnit resourceUnit = new SimpleResourceUnit(broker, new PulsarResourceDescription());
availableBrokers.computeIfAbsent(0L, key -> new TreeSet<>()).add(resourceUnit);
}
log.info("Choosing at random from broker list: [{}]", availableBrokers.values());
@@ -956,7 +954,7 @@ private synchronized ResourceUnit getLeastLoadedBroker(ServiceUnitId serviceUnit
Iterator> candidateIterator = finalCandidates.entries().iterator();
while (candidateIterator.hasNext()) {
Map.Entry candidate = candidateIterator.next();
- String candidateBrokerName = candidate.getValue().getResourceId().replace("http://", "");
+ String candidateBrokerName = candidate.getValue().getResourceId();
if (!activeBrokers.contains(candidateBrokerName)) {
candidateIterator.remove(); // Current candidate points to an inactive broker, so remove it
}
@@ -1005,8 +1003,7 @@ private void updateRanking() {
try {
String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker);
LoadReport lr = loadReports.readLock(key).join().get();
- ResourceUnit ru = new SimpleResourceUnit(String.format("http://%s", lr.getName()),
- fromLoadReport(lr));
+ ResourceUnit ru = new SimpleResourceUnit(lr.getName(), fromLoadReport(lr));
this.currentLoadReports.put(ru, lr);
} catch (Exception e) {
log.warn("Error reading load report from Cache for broker - [{}], [{}]", broker, e);
@@ -1072,13 +1069,13 @@ public LoadReport generateLoadReport() throws Exception {
private LoadReport generateLoadReportForcefully() throws Exception {
synchronized (bundleGainsCache) {
try {
- LoadReport loadReport = new LoadReport(pulsar.getSafeWebServiceAddress(),
+ LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(),
pulsar.getBrokerServiceUrlTls());
loadReport.setProtocols(pulsar.getProtocolDataToAdvertise());
loadReport.setNonPersistentTopicsEnabled(pulsar.getConfiguration().isEnableNonPersistentTopics());
loadReport.setPersistentTopicsEnabled(pulsar.getConfiguration().isEnablePersistentTopics());
- loadReport.setName(pulsar.getLookupServiceAddress());
+ loadReport.setName(pulsar.getBrokerId());
loadReport.setBrokerVersionString(pulsar.getBrokerVersion());
SystemResourceUsage systemResourceUsage = this.getSystemResourceUsage();
@@ -1121,8 +1118,8 @@ private LoadReport generateLoadReportForcefully() throws Exception {
loadReport.setAllocatedMsgRateIn(allocatedQuota.getMsgRateIn());
loadReport.setAllocatedMsgRateOut(allocatedQuota.getMsgRateOut());
- final ResourceUnit resourceUnit = new SimpleResourceUnit(
- String.format("http://%s", loadReport.getName()), fromLoadReport(loadReport));
+ final ResourceUnit resourceUnit =
+ new SimpleResourceUnit(loadReport.getName(), fromLoadReport(loadReport));
Set preAllocatedBundles;
if (resourceUnitRankings.containsKey(resourceUnit)) {
preAllocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles();
@@ -1277,7 +1274,7 @@ private synchronized void updateBrokerToNamespaceToBundle() {
final Set preallocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles();
final ConcurrentOpenHashMap> namespaceToBundleRange =
brokerToNamespaceToBundleRange
- .computeIfAbsent(broker.replace("http://", ""),
+ .computeIfAbsent(broker,
k -> ConcurrentOpenHashMap.>newBuilder()
.build());
@@ -1455,7 +1452,6 @@ public String setNamespaceBundleAffinity(String bundle, String broker) {
if (StringUtils.isBlank(broker)) {
return this.bundleBrokerAffinityMap.remove(bundle);
}
- broker = broker.replaceFirst("http[s]?://", "");
return this.bundleBrokerAffinityMap.put(bundle, broker);
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java
index b92af5b7c69f3..d8dcfa007cfc5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java
@@ -25,7 +25,7 @@
import org.apache.commons.lang3.mutable.MutableDouble;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.mutable.MutableObject;
-import org.apache.commons.lang3.tuple.Triple;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.loadbalance.LoadData;
import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy;
@@ -36,7 +36,7 @@
/**
* This strategy tends to distribute load uniformly across all brokers. This strategy checks load difference between
- * broker with highest load and broker with lowest load. If the difference is higher than configured thresholds
+ * broker with the highest load and broker with the lowest load. If the difference is higher than configured thresholds
* {@link ServiceConfiguration#getLoadBalancerMsgRateDifferenceShedderThreshold()} or
* {@link ServiceConfiguration#getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold()} then it finds out
* bundles which can be unloaded to distribute traffic evenly across all brokers.
@@ -63,25 +63,37 @@ public Multimap findBundlesForUnloading(final LoadData loadData,
Map loadBundleData = loadData.getBundleDataForLoadShedding();
Map recentlyUnloadedBundles = loadData.getRecentlyUnloadedBundles();
- MutableObject overloadedBroker = new MutableObject<>();
- MutableObject underloadedBroker = new MutableObject<>();
+ MutableObject msgRateOverloadedBroker = new MutableObject<>();
+ MutableObject msgThroughputOverloadedBroker = new MutableObject<>();
+ MutableObject msgRateUnderloadedBroker = new MutableObject<>();
+ MutableObject msgThroughputUnderloadedBroker = new MutableObject<>();
MutableDouble maxMsgRate = new MutableDouble(-1);
- MutableDouble maxThroughputRate = new MutableDouble(-1);
+ MutableDouble maxThroughput = new MutableDouble(-1);
MutableDouble minMsgRate = new MutableDouble(Integer.MAX_VALUE);
- MutableDouble minThroughputRate = new MutableDouble(Integer.MAX_VALUE);
+ MutableDouble minThroughput = new MutableDouble(Integer.MAX_VALUE);
+
brokersData.forEach((broker, data) -> {
double msgRate = data.getLocalData().getMsgRateIn() + data.getLocalData().getMsgRateOut();
double throughputRate = data.getLocalData().getMsgThroughputIn()
+ data.getLocalData().getMsgThroughputOut();
- if (msgRate > maxMsgRate.getValue() || throughputRate > maxThroughputRate.getValue()) {
- overloadedBroker.setValue(broker);
+ if (msgRate > maxMsgRate.getValue()) {
+ msgRateOverloadedBroker.setValue(broker);
maxMsgRate.setValue(msgRate);
- maxThroughputRate.setValue(throughputRate);
}
- if (msgRate < minMsgRate.getValue() || throughputRate < minThroughputRate.getValue()) {
- underloadedBroker.setValue(broker);
+
+ if (throughputRate > maxThroughput.getValue()) {
+ msgThroughputOverloadedBroker.setValue(broker);
+ maxThroughput.setValue(throughputRate);
+ }
+
+ if (msgRate < minMsgRate.getValue()) {
+ msgRateUnderloadedBroker.setValue(broker);
minMsgRate.setValue(msgRate);
- minThroughputRate.setValue(throughputRate);
+ }
+
+ if (throughputRate < minThroughput.getValue()) {
+ msgThroughputUnderloadedBroker.setValue(broker);
+ minThroughput.setValue(throughputRate);
}
});
@@ -91,12 +103,12 @@ public Multimap findBundlesForUnloading(final LoadData loadData,
if (minMsgRate.getValue() <= EPS && minMsgRate.getValue() >= -EPS) {
minMsgRate.setValue(1.0);
}
- if (minThroughputRate.getValue() <= EPS && minThroughputRate.getValue() >= -EPS) {
- minThroughputRate.setValue(1.0);
+ if (minThroughput.getValue() <= EPS && minThroughput.getValue() >= -EPS) {
+ minThroughput.setValue(1.0);
}
double msgRateDifferencePercentage = ((maxMsgRate.getValue() - minMsgRate.getValue()) * 100)
/ (minMsgRate.getValue());
- double msgThroughputDifferenceRate = maxThroughputRate.getValue() / minThroughputRate.getValue();
+ double msgThroughputDifferenceRate = maxThroughput.getValue() / minThroughput.getValue();
// if the threshold matches then find out how much load needs to be unloaded by considering number of msgRate
// and throughput.
@@ -105,66 +117,91 @@ public Multimap findBundlesForUnloading(final LoadData loadData,
boolean isMsgThroughputThresholdExceeded = conf
.getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold() > 0
&& msgThroughputDifferenceRate > conf
- .getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold();
+ .getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold();
if (isMsgRateThresholdExceeded || isMsgThroughputThresholdExceeded) {
- if (log.isDebugEnabled()) {
- log.debug(
- "Found bundles for uniform load balancing. "
- + "overloaded broker {} with (msgRate,throughput)= ({},{}) "
- + "and underloaded broker {} with (msgRate,throughput)= ({},{})",
- overloadedBroker.getValue(), maxMsgRate.getValue(), maxThroughputRate.getValue(),
- underloadedBroker.getValue(), minMsgRate.getValue(), minThroughputRate.getValue());
- }
MutableInt msgRateRequiredFromUnloadedBundles = new MutableInt(
(int) ((maxMsgRate.getValue() - minMsgRate.getValue()) * conf.getMaxUnloadPercentage()));
MutableInt msgThroughputRequiredFromUnloadedBundles = new MutableInt(
- (int) ((maxThroughputRate.getValue() - minThroughputRate.getValue())
+ (int) ((maxThroughput.getValue() - minThroughput.getValue())
* conf.getMaxUnloadPercentage()));
- LocalBrokerData overloadedBrokerData = brokersData.get(overloadedBroker.getValue()).getLocalData();
-
- if (overloadedBrokerData.getBundles().size() > 1
- && (msgRateRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessage()
- || msgThroughputRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessageThroughput())) {
- // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with
- // under-loaded broker
- loadBundleData.entrySet().stream()
- .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey()))
- .map((e) -> {
- String bundle = e.getKey();
- BundleData bundleData = e.getValue();
- TimeAverageMessageData shortTermData = bundleData.getShortTermData();
- double throughput = isMsgRateThresholdExceeded
- ? shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut()
- : shortTermData.getMsgThroughputIn() + shortTermData.getMsgThroughputOut();
- return Triple.of(bundle, bundleData, throughput);
- }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft()))
- .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> {
- if (conf.getMaxUnloadBundleNumPerShedding() != -1
- && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) {
- return;
- }
- String bundle = e.getLeft();
- BundleData bundleData = e.getMiddle();
- TimeAverageMessageData shortTermData = bundleData.getShortTermData();
- double throughput = shortTermData.getMsgThroughputIn()
- + shortTermData.getMsgThroughputOut();
- double bundleMsgRate = shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut();
- if (isMsgRateThresholdExceeded) {
+ if (isMsgRateThresholdExceeded) {
+ if (log.isDebugEnabled()) {
+ log.debug("Found bundles for uniform load balancing. "
+ + "msgRate overloaded broker: {} with msgRate: {}, "
+ + "msgRate underloaded broker: {} with msgRate: {}",
+ msgRateOverloadedBroker.getValue(), maxMsgRate.getValue(),
+ msgRateUnderloadedBroker.getValue(), minMsgRate.getValue());
+ }
+ LocalBrokerData overloadedBrokerData =
+ brokersData.get(msgRateOverloadedBroker.getValue()).getLocalData();
+ if (overloadedBrokerData.getBundles().size() > 1
+ && (msgRateRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessage())) {
+ // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with
+ // under-loaded broker
+ loadBundleData.entrySet().stream()
+ .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey()))
+ .map((e) -> {
+ String bundle = e.getKey();
+ TimeAverageMessageData shortTermData = e.getValue().getShortTermData();
+ double msgRate = shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut();
+ return Pair.of(bundle, msgRate);
+ }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft()))
+ .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> {
+ if (conf.getMaxUnloadBundleNumPerShedding() != -1
+ && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) {
+ return;
+ }
+ String bundle = e.getLeft();
+ double bundleMsgRate = e.getRight();
if (bundleMsgRate <= (msgRateRequiredFromUnloadedBundles.getValue()
+ 1000/* delta */)) {
log.info("Found bundle to unload with msgRate {}", bundleMsgRate);
msgRateRequiredFromUnloadedBundles.add(-bundleMsgRate);
- selectedBundlesCache.put(overloadedBroker.getValue(), bundle);
+ selectedBundlesCache.put(msgRateOverloadedBroker.getValue(), bundle);
}
- } else {
- if (throughput <= (msgThroughputRequiredFromUnloadedBundles.getValue())) {
- log.info("Found bundle to unload with throughput {}", throughput);
- msgThroughputRequiredFromUnloadedBundles.add(-throughput);
- selectedBundlesCache.put(overloadedBroker.getValue(), bundle);
+ });
+ }
+ } else {
+ if (log.isDebugEnabled()) {
+ log.debug("Found bundles for uniform load balancing. "
+ + "msgThroughput overloaded broker: {} with msgThroughput {}, "
+ + "msgThroughput underloaded broker: {} with msgThroughput: {}",
+ msgThroughputOverloadedBroker.getValue(), maxThroughput.getValue(),
+ msgThroughputUnderloadedBroker.getValue(), minThroughput.getValue());
+ }
+ LocalBrokerData overloadedBrokerData =
+ brokersData.get(msgThroughputOverloadedBroker.getValue()).getLocalData();
+ if (overloadedBrokerData.getBundles().size() > 1
+ &&
+ msgThroughputRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessageThroughput()) {
+ // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with
+ // under-loaded broker
+ loadBundleData.entrySet().stream()
+ .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey()))
+ .map((e) -> {
+ String bundle = e.getKey();
+ TimeAverageMessageData shortTermData = e.getValue().getShortTermData();
+ double msgThroughput = shortTermData.getMsgThroughputIn()
+ + shortTermData.getMsgThroughputOut();
+ return Pair.of(bundle, msgThroughput);
+ }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft()))
+ .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> {
+ if (conf.getMaxUnloadBundleNumPerShedding() != -1
+ && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) {
+ return;
}
- }
- });
+ String bundle = e.getLeft();
+ double msgThroughput = e.getRight();
+ if (msgThroughput <= (msgThroughputRequiredFromUnloadedBundles.getValue()
+ + 1000/* delta */)) {
+ log.info("Found bundle to unload with msgThroughput {}", msgThroughput);
+ msgThroughputRequiredFromUnloadedBundles.add(-msgThroughput);
+ selectedBundlesCache.put(msgThroughputOverloadedBroker.getValue(), bundle);
+ }
+ });
+ }
+
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java
index bd70201cba55d..7b2c777414884 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java
@@ -26,11 +26,11 @@
import java.net.URISyntaxException;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
import javax.ws.rs.Encoded;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
@@ -48,6 +48,7 @@
import org.apache.pulsar.common.policies.data.TopicOperation;
import org.apache.pulsar.common.util.Codec;
import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,7 +57,7 @@ public class TopicLookupBase extends PulsarWebResource {
private static final String LOOKUP_PATH_V1 = "/lookup/v2/destination/";
private static final String LOOKUP_PATH_V2 = "/lookup/v2/topic/";
- protected CompletableFuture internalLookupTopicAsync(TopicName topicName, boolean authoritative,
+ protected CompletableFuture internalLookupTopicAsync(final TopicName topicName, boolean authoritative,
String listenerName) {
if (!pulsar().getBrokerService().getLookupRequestSemaphore().tryAcquire()) {
log.warn("No broker was found available for topic {}", topicName);
@@ -79,7 +80,8 @@ protected CompletableFuture internalLookupTopicAsync(TopicName topic
})
.thenCompose(exist -> {
if (!exist) {
- throw new RestException(Response.Status.NOT_FOUND, "Topic not found.");
+ throw new RestException(Response.Status.NOT_FOUND,
+ String.format("Topic not found %s", topicName.toString()));
}
CompletableFuture> lookupFuture = pulsar().getNamespaceService()
.getBrokerServiceUrlAsync(topicName,
@@ -131,10 +133,10 @@ protected CompletableFuture internalLookupTopicAsync(TopicName topic
pulsar().getBrokerService().getLookupRequestSemaphore().release();
return result.getLookupData();
}
- }).exceptionally(ex->{
- pulsar().getBrokerService().getLookupRequestSemaphore().release();
- throw FutureUtil.wrapToCompletionException(ex);
});
+ }).exceptionally(ex -> {
+ pulsar().getBrokerService().getLookupRequestSemaphore().release();
+ throw FutureUtil.wrapToCompletionException(ex);
});
}
@@ -318,35 +320,40 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe
requestId, shouldRedirectThroughServiceUrl(conf, lookupData)));
}
}).exceptionally(ex -> {
- if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
- log.info("Failed to lookup {} for topic {} with error {}", clientAppId,
- topicName.toString(), ex.getCause().getMessage());
- } else {
- log.warn("Failed to lookup {} for topic {} with error {}", clientAppId,
- topicName.toString(), ex.getMessage(), ex);
- }
- lookupfuture.complete(
- newLookupErrorResponse(ServerError.ServiceNotReady, ex.getMessage(), requestId));
- return null;
- });
+ handleLookupError(lookupfuture, topicName.toString(), clientAppId, requestId, ex);
+ return null;
+ });
}
-
}).exceptionally(ex -> {
- if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
- log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
- ex.getCause().getMessage());
- } else {
- log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
- ex.getMessage(), ex);
- }
-
- lookupfuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady, ex.getMessage(), requestId));
+ handleLookupError(lookupfuture, topicName.toString(), clientAppId, requestId, ex);
return null;
});
return lookupfuture;
}
+ private static void handleLookupError(CompletableFuture lookupFuture, String topicName, String clientAppId,
+ long requestId, Throwable ex){
+ Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex);
+ final String errorMsg = unwrapEx.getMessage();
+ if (unwrapEx instanceof PulsarServerException) {
+ unwrapEx = FutureUtil.unwrapCompletionException(unwrapEx.getCause());
+ }
+ if (unwrapEx instanceof IllegalStateException) {
+ // Current broker still hold the bundle's lock, but the bundle is being unloading.
+ log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg);
+ lookupFuture.complete(newLookupErrorResponse(ServerError.MetadataError, errorMsg, requestId));
+ } else if (unwrapEx instanceof MetadataStoreException) {
+ // Load bundle ownership or acquire lock failed.
+ // Differ with "IllegalStateException", print warning log.
+ log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg);
+ lookupFuture.complete(newLookupErrorResponse(ServerError.MetadataError, errorMsg, requestId));
+ } else {
+ log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg);
+ lookupFuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady, errorMsg, requestId));
+ }
+ }
+
protected TopicName getTopicName(String topicDomain, String tenant, String cluster, String namespace,
@Encoded String encodedTopic) {
String decodedName = Codec.decode(encodedTopic);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
index 585d62c5b1f27..61e045ed304fd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
@@ -133,9 +133,9 @@ public class NamespaceService implements AutoCloseable {
public static final Pattern HEARTBEAT_NAMESPACE_PATTERN = Pattern.compile("pulsar/[^/]+/([^:]+:\\d+)");
public static final Pattern HEARTBEAT_NAMESPACE_PATTERN_V2 = Pattern.compile("pulsar/([^:]+:\\d+)");
public static final Pattern SLA_NAMESPACE_PATTERN = Pattern.compile(SLA_NAMESPACE_PROPERTY + "/[^/]+/([^:]+:\\d+)");
- public static final String HEARTBEAT_NAMESPACE_FMT = "pulsar/%s/%s:%s";
- public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s:%s";
- public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s:%s";
+ public static final String HEARTBEAT_NAMESPACE_FMT = "pulsar/%s/%s";
+ public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s";
+ public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s";
private final ConcurrentOpenHashMap namespaceClients;
@@ -164,7 +164,7 @@ public class NamespaceService implements AutoCloseable {
*/
public NamespaceService(PulsarService pulsar) {
this.pulsar = pulsar;
- host = pulsar.getAdvertisedAddress();
+ this.host = pulsar.getAdvertisedAddress();
this.config = pulsar.getConfiguration();
this.loadManager = pulsar.getLoadManager();
this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32());
@@ -189,10 +189,10 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN
CompletableFuture> future = getBundleAsync(topic)
.thenCompose(bundle -> {
// Do redirection if the cluster is in rollback or deploying.
- return redirectManager.findRedirectLookupResultAsync().thenCompose(optResult -> {
+ return findRedirectLookupResultAsync(bundle).thenCompose(optResult -> {
if (optResult.isPresent()) {
LOG.info("[{}] Redirect lookup request to {} for topic {}",
- pulsar.getSafeWebServiceAddress(), optResult.get(), topic);
+ pulsar.getBrokerId(), optResult.get(), topic);
return CompletableFuture.completedFuture(optResult);
}
if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) {
@@ -221,6 +221,13 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN
return future;
}
+ private CompletableFuture> findRedirectLookupResultAsync(ServiceUnitId bundle) {
+ if (isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) {
+ return CompletableFuture.completedFuture(Optional.empty());
+ }
+ return redirectManager.findRedirectLookupResultAsync();
+ }
+
public CompletableFuture getBundleAsync(TopicName topic) {
return bundleFactory.getBundlesAsync(topic.getNamespaceObject())
.thenApply(bundles -> bundles.findBundle(topic));
@@ -288,11 +295,10 @@ public Optional getWebServiceUrl(ServiceUnitId suName, LookupOptions option
private CompletableFuture> internalGetWebServiceUrl(@Nullable ServiceUnitId topic,
NamespaceBundle bundle,
LookupOptions options) {
-
- return redirectManager.findRedirectLookupResultAsync().thenCompose(optResult -> {
+ return findRedirectLookupResultAsync(bundle).thenCompose(optResult -> {
if (optResult.isPresent()) {
LOG.info("[{}] Redirect lookup request to {} for topic {}",
- pulsar.getSafeWebServiceAddress(), optResult.get(), topic);
+ pulsar.getBrokerId(), optResult.get(), topic);
try {
LookupData lookupData = optResult.get().getLookupData();
final String redirectUrl = options.isRequestHttps()
@@ -332,15 +338,17 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv
* @throws PulsarServerException if an unexpected error occurs
*/
public void registerBootstrapNamespaces() throws PulsarServerException {
-
+ String brokerId = pulsar.getBrokerId();
// ensure that we own the heartbeat namespace
- if (registerNamespace(getHeartbeatNamespace(host, config), true)) {
- LOG.info("added heartbeat namespace name in local cache: ns={}", getHeartbeatNamespace(host, config));
+ if (registerNamespace(getHeartbeatNamespace(brokerId, config), true)) {
+ LOG.info("added heartbeat namespace name in local cache: ns={}",
+ getHeartbeatNamespace(brokerId, config));
}
// ensure that we own the heartbeat namespace
- if (registerNamespace(getHeartbeatNamespaceV2(host, config), true)) {
- LOG.info("added heartbeat namespace name in local cache: ns={}", getHeartbeatNamespaceV2(host, config));
+ if (registerNamespace(getHeartbeatNamespaceV2(brokerId, config), true)) {
+ LOG.info("added heartbeat namespace name in local cache: ns={}",
+ getHeartbeatNamespaceV2(brokerId, config));
}
// we may not need strict ownership checking for bootstrap names for now
@@ -498,7 +506,6 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
return;
}
String candidateBroker;
- String candidateBrokerAdvertisedAddr = null;
LeaderElectionService les = pulsar.getLeaderElectionService();
if (les == null) {
@@ -533,14 +540,14 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
if (options.isAuthoritative()) {
// leader broker already assigned the current broker as owner
- candidateBroker = pulsar.getSafeWebServiceAddress();
+ candidateBroker = pulsar.getBrokerId();
} else {
LoadManager loadManager = this.loadManager.get();
boolean makeLoadManagerDecisionOnThisBroker = !loadManager.isCentralized() || les.isLeader();
if (!makeLoadManagerDecisionOnThisBroker) {
// If leader is not active, fallback to pick the least loaded from current broker loadmanager
boolean leaderBrokerActive = currentLeader.isPresent()
- && isBrokerActive(currentLeader.get().getServiceUrl());
+ && isBrokerActive(currentLeader.get().getBrokerId());
if (!leaderBrokerActive) {
makeLoadManagerDecisionOnThisBroker = true;
if (currentLeader.isEmpty()) {
@@ -559,7 +566,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
}
}
if (makeLoadManagerDecisionOnThisBroker) {
- Optional> availableBroker = getLeastLoadedFromLoadManager(bundle);
+ Optional availableBroker = getLeastLoadedFromLoadManager(bundle);
if (availableBroker.isEmpty()) {
LOG.warn("Load manager didn't return any available broker. "
+ "Returning empty result to lookup. NamespaceBundle[{}]",
@@ -567,12 +574,11 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
lookupFuture.complete(Optional.empty());
return;
}
- candidateBroker = availableBroker.get().getLeft();
- candidateBrokerAdvertisedAddr = availableBroker.get().getRight();
+ candidateBroker = availableBroker.get();
authoritativeRedirect = true;
} else {
// forward to leader broker to make assignment
- candidateBroker = currentLeader.get().getServiceUrl();
+ candidateBroker = currentLeader.get().getBrokerId();
}
}
}
@@ -585,7 +591,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
try {
Objects.requireNonNull(candidateBroker);
- if (candidateBroker.equals(pulsar.getSafeWebServiceAddress())) {
+ if (candidateBroker.equals(pulsar.getBrokerId())) {
// Load manager decided that the local broker should try to become the owner
ownershipCache.tryAcquiringOwnership(bundle).thenAccept(ownerInfo -> {
if (ownerInfo.isDisabled()) {
@@ -636,8 +642,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle,
}
// Now setting the redirect url
- createLookupResult(candidateBrokerAdvertisedAddr == null ? candidateBroker
- : candidateBrokerAdvertisedAddr, authoritativeRedirect, options.getAdvertisedListenerName())
+ createLookupResult(candidateBroker, authoritativeRedirect, options.getAdvertisedListenerName())
.thenAccept(lookupResult -> lookupFuture.complete(Optional.of(lookupResult)))
.exceptionally(ex -> {
lookupFuture.completeExceptionally(ex);
@@ -657,7 +662,7 @@ public CompletableFuture createLookupResult(String candidateBroker
CompletableFuture lookupFuture = new CompletableFuture<>();
try {
checkArgument(StringUtils.isNotBlank(candidateBroker), "Lookup broker can't be null %s", candidateBroker);
- String path = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + parseHostAndPort(candidateBroker);
+ String path = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + candidateBroker;
localBrokerDataCache.get(path).thenAccept(reportData -> {
if (reportData.isPresent()) {
@@ -693,30 +698,20 @@ public CompletableFuture createLookupResult(String candidateBroker
return lookupFuture;
}
- private boolean isBrokerActive(String candidateBroker) {
- String candidateBrokerHostAndPort = parseHostAndPort(candidateBroker);
+ public boolean isBrokerActive(String candidateBroker) {
Set availableBrokers = getAvailableBrokers();
- if (availableBrokers.contains(candidateBrokerHostAndPort)) {
+ if (availableBrokers.contains(candidateBroker)) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Broker {} ({}) is available for.", candidateBroker, candidateBrokerHostAndPort);
+ LOG.debug("Broker {} is available for.", candidateBroker);
}
return true;
} else {
- LOG.warn("Broker {} ({}) couldn't be found in available brokers {}",
- candidateBroker, candidateBrokerHostAndPort,
- String.join(",", availableBrokers));
+ LOG.warn("Broker {} couldn't be found in available brokers {}",
+ candidateBroker, String.join(",", availableBrokers));
return false;
}
}
- private static String parseHostAndPort(String candidateBroker) {
- int uriSeparatorPos = candidateBroker.indexOf("://");
- if (uriSeparatorPos == -1) {
- throw new IllegalArgumentException("'" + candidateBroker + "' isn't an URI.");
- }
- return candidateBroker.substring(uriSeparatorPos + 3);
- }
-
private Set getAvailableBrokers() {
try {
return loadManager.get().getAvailableBrokers();
@@ -732,7 +727,7 @@ private Set getAvailableBrokers() {
* @return the least loaded broker addresses
* @throws Exception if an error occurs
*/
- private Optional> getLeastLoadedFromLoadManager(ServiceUnitId serviceUnit) throws Exception {
+ private Optional getLeastLoadedFromLoadManager(ServiceUnitId serviceUnit) throws Exception {
Optional leastLoadedBroker = loadManager.get().getLeastLoaded(serviceUnit);
if (leastLoadedBroker.isEmpty()) {
LOG.warn("No broker is available for {}", serviceUnit);
@@ -740,15 +735,13 @@ private Optional> getLeastLoadedFromLoadManager(ServiceUnit
}
String lookupAddress = leastLoadedBroker.get().getResourceId();
- String advertisedAddr = (String) leastLoadedBroker.get()
- .getProperty(ResourceUnit.PROPERTY_KEY_BROKER_ZNODE_NAME);
if (LOG.isDebugEnabled()) {
LOG.debug("{} : redirecting to the least loaded broker, lookup address={}",
- pulsar.getSafeWebServiceAddress(),
+ pulsar.getBrokerId(),
lookupAddress);
}
- return Optional.of(Pair.of(lookupAddress, advertisedAddr));
+ return Optional.of(lookupAddress);
}
public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle) {
@@ -1441,6 +1434,9 @@ private CompletableFuture> getPartitionsForTopic(TopicName topicNam
});
}
+ /***
+ * List persistent topics names under a namespace, the topic name contains the partition suffix.
+ */
public CompletableFuture> getListOfPersistentTopics(NamespaceName namespaceName) {
return pulsar.getPulsarResources().getTopicResources().listPersistentTopicsAsync(namespaceName);
}
@@ -1550,10 +1546,6 @@ public boolean checkOwnershipPresent(NamespaceBundle bundle) throws Exception {
public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bundle) {
if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) {
- if (bundle.getNamespaceObject().equals(SYSTEM_NAMESPACE)) {
- return FutureUtil.failedFuture(new UnsupportedOperationException(
- "Ownership check for system namespace is not supported"));
- }
ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get());
return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle)
.thenApply(Optional::isPresent);
@@ -1562,7 +1554,7 @@ public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bun
}
public void unloadSLANamespace() throws Exception {
- NamespaceName namespaceName = getSLAMonitorNamespace(host, config);
+ NamespaceName namespaceName = getSLAMonitorNamespace(pulsar.getBrokerId(), config);
LOG.info("Checking owner for SLA namespace {}", namespaceName);
@@ -1579,41 +1571,23 @@ public void unloadSLANamespace() throws Exception {
LOG.info("Namespace {} unloaded successfully", namespaceName);
}
- public static NamespaceName getHeartbeatNamespace(String host, ServiceConfiguration config) {
- Integer port = null;
- if (config.getWebServicePort().isPresent()) {
- port = config.getWebServicePort().get();
- } else if (config.getWebServicePortTls().isPresent()) {
- port = config.getWebServicePortTls().get();
- }
- return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), host, port));
+ public static NamespaceName getHeartbeatNamespace(String lookupBroker, ServiceConfiguration config) {
+ return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), lookupBroker));
}
- public static NamespaceName getHeartbeatNamespaceV2(String host, ServiceConfiguration config) {
- Integer port = null;
- if (config.getWebServicePort().isPresent()) {
- port = config.getWebServicePort().get();
- } else if (config.getWebServicePortTls().isPresent()) {
- port = config.getWebServicePortTls().get();
- }
- return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, host, port));
+ public static NamespaceName getHeartbeatNamespaceV2(String lookupBroker, ServiceConfiguration config) {
+ return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, lookupBroker));
}
- public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) {
- Integer port = null;
- if (config.getWebServicePort().isPresent()) {
- port = config.getWebServicePort().get();
- } else if (config.getWebServicePortTls().isPresent()) {
- port = config.getWebServicePortTls().get();
- }
- return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port));
+ public static NamespaceName getSLAMonitorNamespace(String lookupBroker, ServiceConfiguration config) {
+ return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), lookupBroker));
}
public static String checkHeartbeatNamespace(ServiceUnitId ns) {
Matcher m = HEARTBEAT_NAMESPACE_PATTERN.matcher(ns.getNamespaceObject().toString());
if (m.matches()) {
LOG.debug("Heartbeat namespace matched the lookup namespace {}", ns.getNamespaceObject().toString());
- return String.format("http://%s", m.group(1));
+ return m.group(1);
} else {
return null;
}
@@ -1623,7 +1597,7 @@ public static String checkHeartbeatNamespaceV2(ServiceUnitId ns) {
Matcher m = HEARTBEAT_NAMESPACE_PATTERN_V2.matcher(ns.getNamespaceObject().toString());
if (m.matches()) {
LOG.debug("Heartbeat namespace v2 matched the lookup namespace {}", ns.getNamespaceObject().toString());
- return String.format("http://%s", m.group(1));
+ return m.group(1);
} else {
return null;
}
@@ -1632,7 +1606,7 @@ public static String checkHeartbeatNamespaceV2(ServiceUnitId ns) {
public static String getSLAMonitorBrokerName(ServiceUnitId ns) {
Matcher m = SLA_NAMESPACE_PATTERN.matcher(ns.getNamespaceObject().toString());
if (m.matches()) {
- return String.format("http://%s", m.group(1));
+ return m.group(1);
} else {
return null;
}
@@ -1650,7 +1624,7 @@ public static boolean isSystemServiceNamespace(String namespace) {
* @param namespace the namespace name
* @return True if namespace is HEARTBEAT_NAMESPACE or SLA_NAMESPACE
*/
- public static boolean filterNamespaceForShedding(String namespace) {
+ public static boolean isSLAOrHeartbeatNamespace(String namespace) {
return SLA_NAMESPACE_PATTERN.matcher(namespace).matches()
|| HEARTBEAT_NAMESPACE_PATTERN.matcher(namespace).matches()
|| HEARTBEAT_NAMESPACE_PATTERN_V2.matcher(namespace).matches();
@@ -1663,14 +1637,16 @@ public static boolean isHeartbeatNamespace(ServiceUnitId ns) {
}
public boolean registerSLANamespace() throws PulsarServerException {
- boolean isNameSpaceRegistered = registerNamespace(getSLAMonitorNamespace(host, config), false);
+ String brokerId = pulsar.getBrokerId();
+ boolean isNameSpaceRegistered = registerNamespace(getSLAMonitorNamespace(brokerId, config), false);
if (isNameSpaceRegistered) {
if (LOG.isDebugEnabled()) {
LOG.debug("Added SLA Monitoring namespace name in local cache: ns={}",
- getSLAMonitorNamespace(host, config));
+ getSLAMonitorNamespace(brokerId, config));
}
} else if (LOG.isDebugEnabled()) {
- LOG.debug("SLA Monitoring not owned by the broker: ns={}", getSLAMonitorNamespace(host, config));
+ LOG.debug("SLA Monitoring not owned by the broker: ns={}",
+ getSLAMonitorNamespace(brokerId, config));
}
return isNameSpaceRegistered;
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java
index 86003153714cb..0033abf36c78c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java
@@ -122,10 +122,10 @@ public OwnershipCache(PulsarService pulsar, NamespaceBundleFactory bundleFactory
this.ownerBrokerUrl = pulsar.getBrokerServiceUrl();
this.ownerBrokerUrlTls = pulsar.getBrokerServiceUrlTls();
this.selfOwnerInfo = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls,
- pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(),
+ pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(),
false, pulsar.getAdvertisedListeners());
this.selfOwnerInfoDisabled = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls,
- pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(),
+ pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(),
true, pulsar.getAdvertisedListeners());
this.lockManager = pulsar.getCoordinationService().getLockManager(NamespaceEphemeralData.class);
this.locallyAcquiredLocks = new ConcurrentHashMap<>();
@@ -336,7 +336,7 @@ public Map> getLocallyAcqu
public synchronized boolean refreshSelfOwnerInfo() {
this.selfOwnerInfo = new NamespaceEphemeralData(pulsar.getBrokerServiceUrl(),
- pulsar.getBrokerServiceUrlTls(), pulsar.getSafeWebServiceAddress(),
+ pulsar.getBrokerServiceUrlTls(), pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(), false, pulsar.getAdvertisedListeners());
return selfOwnerInfo.getNativeUrl() != null || selfOwnerInfo.getNativeUrlTls() != null;
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java
index c86aac5316fb9..432aa29798ebd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java
@@ -36,7 +36,7 @@ public final class ServiceUnitUtils {
*/
private static final String OWNER_INFO_ROOT = "/namespace";
- static String path(NamespaceBundle suname) {
+ public static String path(NamespaceBundle suname) {
// The ephemeral node path for new namespaces should always have bundle name appended
return OWNER_INFO_ROOT + "/" + suname.toString();
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlers.java
index 42a82b2de762b..4059ccf5f26eb 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlers.java
@@ -24,6 +24,7 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
@@ -51,6 +52,9 @@ public class ProtocolHandlers implements AutoCloseable {
* @return the collection of protocol handlers
*/
public static ProtocolHandlers load(ServiceConfiguration conf) throws IOException {
+ if (conf.getMessagingProtocols().isEmpty()) {
+ return new ProtocolHandlers(Collections.emptyMap());
+ }
ProtocolHandlerDefinitions definitions =
ProtocolHandlerUtils.searchForHandlers(
conf.getProtocolHandlerDirectory(), conf.getNarExtractionDirectory());
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java
index 85e00bb2f87dc..0377ec86488d4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java
@@ -135,7 +135,7 @@ public void update(long publishRateInMsgs, long publishRateInBytes) {
public boolean tryAcquire(int numbers, long bytes) {
return (publishRateLimiterOnMessage == null || publishRateLimiterOnMessage.tryAcquire(numbers))
- && (publishRateLimiterOnByte == null || publishRateLimiterOnByte.tryAcquire(bytes));
+ && (publishRateLimiterOnByte == null || publishRateLimiterOnByte.tryAcquire(bytes));
}
public void registerRateLimitFunction(String name, RateLimitFunction func) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java
index 5ab81fcbbff27..20c35b4f7769c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java
@@ -54,6 +54,7 @@
import org.apache.avro.io.DecoderFactory;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase;
import org.apache.pulsar.broker.authentication.AuthenticationParameters;
@@ -433,8 +434,10 @@ private CompletableFuture lookUpBrokerForTopic(TopicName partitionedTopicN
}
LookupResult result = optionalResult.get();
- if (result.getLookupData().getHttpUrl().equals(pulsar().getWebServiceAddress())
- || result.getLookupData().getHttpUrlTls().equals(pulsar().getWebServiceAddressTls())) {
+ String httpUrl = result.getLookupData().getHttpUrl();
+ String httpUrlTls = result.getLookupData().getHttpUrlTls();
+ if ((StringUtils.isNotBlank(httpUrl) && httpUrl.equals(pulsar().getWebServiceAddress()))
+ || (StringUtils.isNotBlank(httpUrlTls) && httpUrlTls.equals(pulsar().getWebServiceAddressTls()))) {
// Current broker owns the topic, add to owning topic.
if (log.isDebugEnabled()) {
log.debug("Complete topic look up for rest produce message request for topic {}, "
@@ -455,12 +458,10 @@ private CompletableFuture lookUpBrokerForTopic(TopicName partitionedTopicN
}
if (result.isRedirect()) {
// Redirect lookup.
- completeLookup(Pair.of(Arrays.asList(result.getLookupData().getHttpUrl(),
- result.getLookupData().getHttpUrlTls()), false), redirectAddresses, future);
+ completeLookup(Pair.of(Arrays.asList(httpUrl, httpUrlTls), false), redirectAddresses, future);
} else {
// Found owner for topic.
- completeLookup(Pair.of(Arrays.asList(result.getLookupData().getHttpUrl(),
- result.getLookupData().getHttpUrlTls()), true), redirectAddresses, future);
+ completeLookup(Pair.of(Arrays.asList(httpUrl, httpUrlTls), true), redirectAddresses, future);
}
}
}).exceptionally(exception -> {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
index 50f05f80e3f8b..0497a72acc64d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
@@ -49,6 +49,7 @@
import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.Markers;
+import org.apache.pulsar.compaction.Compactor;
import org.checkerframework.checker.nullness.qual.Nullable;
@Slf4j
@@ -174,34 +175,46 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
if (msgMetadata != null && msgMetadata.hasTxnidMostBits()
&& msgMetadata.hasTxnidLeastBits()) {
if (Markers.isTxnMarker(msgMetadata)) {
- // because consumer can receive message is smaller than maxReadPosition,
- // so this marker is useless for this subscription
- individualAcknowledgeMessageIfNeeded(entry.getPosition(), Collections.emptyMap());
- entries.set(i, null);
- entry.release();
- continue;
+ if (cursor == null || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) {
+ // because consumer can receive message is smaller than maxReadPosition,
+ // so this marker is useless for this subscription
+ individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()),
+ Collections.emptyMap());
+ entries.set(i, null);
+ entry.release();
+ continue;
+ }
} else if (((PersistentTopic) subscription.getTopic())
.isTxnAborted(new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()),
(PositionImpl) entry.getPosition())) {
- individualAcknowledgeMessageIfNeeded(entry.getPosition(), Collections.emptyMap());
+ individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()),
+ Collections.emptyMap());
entries.set(i, null);
entry.release();
continue;
}
}
- if (msgMetadata == null || Markers.isServerOnlyMarker(msgMetadata)) {
+ if (msgMetadata == null || (Markers.isServerOnlyMarker(msgMetadata))) {
PositionImpl pos = (PositionImpl) entry.getPosition();
// Message metadata was corrupted or the messages was a server-only marker
if (Markers.isReplicatedSubscriptionSnapshotMarker(msgMetadata)) {
+ final int readerIndex = metadataAndPayload.readerIndex();
processReplicatedSubscriptionSnapshot(pos, metadataAndPayload);
+ metadataAndPayload.readerIndex(readerIndex);
}
- entries.set(i, null);
- entry.release();
- individualAcknowledgeMessageIfNeeded(pos, Collections.emptyMap());
- continue;
+ // Deliver marker to __compaction cursor to avoid compaction task stuck,
+ // and filter out them when doing topic compaction.
+ if (msgMetadata == null || cursor == null
+ || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) {
+ entries.set(i, null);
+ entry.release();
+ individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos),
+ Collections.emptyMap());
+ continue;
+ }
} else if (trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) {
// The message is marked for delayed delivery. Ignore for now.
entries.set(i, null);
@@ -213,12 +226,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
this.filterAcceptedMsgs.add(entryMsgCnt);
}
- totalEntries++;
int batchSize = msgMetadata.getNumMessagesInBatch();
- totalMessages += batchSize;
- totalBytes += metadataAndPayload.readableBytes();
- totalChunkedMessages += msgMetadata.hasChunkId() ? 1 : 0;
- batchSizes.setBatchSize(i, batchSize);
long[] ackSet = null;
if (indexesAcks != null && cursor != null) {
PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId());
@@ -262,6 +270,12 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
}
}
+ totalEntries++;
+ totalMessages += batchSize;
+ totalBytes += metadataAndPayload.readableBytes();
+ totalChunkedMessages += msgMetadata.hasChunkId() ? 1 : 0;
+ batchSizes.setBatchSize(i, batchSize);
+
BrokerInterceptor interceptor = subscription.interceptor();
if (null != interceptor) {
// keep for compatibility if users has implemented the old interface
@@ -270,8 +284,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
}
}
if (CollectionUtils.isNotEmpty(entriesToFiltered)) {
- subscription.acknowledgeMessage(entriesToFiltered, AckType.Individual,
- Collections.emptyMap());
+ individualAcknowledgeMessageIfNeeded(entriesToFiltered, Collections.emptyMap());
int filtered = entriesToFiltered.size();
Topic topic = subscription.getTopic();
@@ -300,9 +313,9 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
return totalEntries;
}
- private void individualAcknowledgeMessageIfNeeded(Position position, Map properties) {
+ private void individualAcknowledgeMessageIfNeeded(List positions, Map properties) {
if (!(subscription instanceof PulsarCompactorSubscription)) {
- subscription.acknowledgeMessage(Collections.singletonList(position), AckType.Individual, properties);
+ subscription.acknowledgeMessage(positions, AckType.Individual, properties);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
index 6dd296d16b53b..1b5b2824257b0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
@@ -194,7 +194,7 @@ protected synchronized CompletableFuture closeProducerAsync() {
return CompletableFuture.completedFuture(null);
}
CompletableFuture future = producer.closeAsync();
- future.thenRun(() -> {
+ return future.thenRun(() -> {
STATE_UPDATER.set(this, State.Stopped);
this.producer = null;
// deactivate further read
@@ -209,7 +209,6 @@ protected synchronized CompletableFuture closeProducerAsync() {
brokerService.executor().schedule(this::closeProducerAsync, waitTimeMs, TimeUnit.MILLISECONDS);
return null;
});
- return future;
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
index b15f8cbf0b848..d5d7e7ade5078 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
@@ -58,7 +58,6 @@
import org.apache.pulsar.broker.service.BrokerServiceException.TopicMigratedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException;
import org.apache.pulsar.broker.service.plugin.EntryFilter;
-import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage;
import org.apache.pulsar.broker.service.schema.SchemaRegistryService;
import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException;
import org.apache.pulsar.broker.stats.prometheus.metrics.Summary;
@@ -674,21 +673,7 @@ private boolean allowAutoUpdateSchema() {
@Override
public CompletableFuture deleteSchema() {
- String id = getSchemaId();
- SchemaRegistryService schemaRegistryService = brokerService.pulsar().getSchemaRegistryService();
- return BookkeeperSchemaStorage.ignoreUnrecoverableBKException(schemaRegistryService.getSchema(id))
- .thenCompose(schema -> {
- if (schema != null) {
- // It's different from `SchemasResource.deleteSchema`
- // because when we delete a topic, the schema
- // history is meaningless. But when we delete a schema of a topic, a new schema could be
- // registered in the future.
- log.info("Delete schema storage of id: {}", id);
- return schemaRegistryService.deleteSchemaStorage(id);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- });
+ return brokerService.deleteSchema(TopicName.get(getName()));
}
@Override
@@ -718,15 +703,14 @@ public CompletableFuture