diff --git a/bin/pulsar-perf b/bin/pulsar-perf index b64b046f10440..5794da2b35013 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -73,9 +73,14 @@ pulsar_help() { cat < where command is one of: - produce Run a producer - consume Run a consumer - help This help message + produce Run a producer + consume Run a consumer + monitor-simple-load-manager Continuously receive broker data when using SimpleLoadManagerImpl + monitor-modular-load-manager Continuously receive broker data when using ModularLoadManagerImpl + simulation-client Run a simulation server acting as a Pulsar client + simulation-controller Run a simulation controller to give commands to servers + + help This help message or command is the full name of a class with a defined main() method. @@ -137,6 +142,14 @@ if [ "$COMMAND" == "produce" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.PerformanceProducer --conf-file $PULSAR_PERFTEST_CONF "$@" elif [ "$COMMAND" == "consume" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.PerformanceConsumer --conf-file $PULSAR_PERFTEST_CONF "$@" +elif [ "$COMMAND" == "monitor-simple-load-manager" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.SimpleLoadManagerBrokerMonitor "$@" +elif [ "$COMMAND" == "monitor-modular-load-manager" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.ModularLoadManagerBrokerMonitor "$@" +elif [ "$COMMAND" == "simulation-client" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.LoadSimulationClient "$@" +elif [ "$COMMAND" == "simulation-controller" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.LoadSimulationController "$@" elif [ "$COMMAND" == "help" ]; then pulsar_help; else diff --git a/conf/broker.conf b/conf/broker.conf index faa32f028bb41..9737beb449417 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -296,3 +296,6 @@ keepAliveIntervalSeconds=30 # How often broker checks for inactive topics to be deleted (topics with no subscriptions and no one connected) brokerServicePurgeInactiveFrequencyInSeconds=60 + +# Name of load manager to use +loadManagerClassName=com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl diff --git a/pulsar-broker-common/src/main/java/com/yahoo/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/com/yahoo/pulsar/broker/ServiceConfiguration.java index c099f7c749fbb..e7691d607a234 100644 --- a/pulsar-broker-common/src/main/java/com/yahoo/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/com/yahoo/pulsar/broker/ServiceConfiguration.java @@ -30,7 +30,7 @@ * Pulsar service configuration object. * */ -public class ServiceConfiguration implements PulsarConfiguration{ +public class ServiceConfiguration implements PulsarConfiguration { /***** --- pulsar configuration --- ****/ // Zookeeper quorum connection string @@ -255,6 +255,9 @@ public class ServiceConfiguration implements PulsarConfiguration{ private int brokerServicePurgeInactiveFrequencyInSeconds = 60; private List bootstrapNamespaces = new ArrayList(); private Properties properties = new Properties(); + // Name of load manager to use + @FieldContext(dynamic = true) + private String loadManagerClassName = "com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl"; public String getZookeeperServers() { return zookeeperServers; @@ -952,4 +955,12 @@ public String getReplicatorPrefix() { public void setReplicatorPrefix(String replicatorPrefix) { this.replicatorPrefix = replicatorPrefix; } + + public String getLoadManagerClassName() { + return loadManagerClassName; + } + + public void setLoadManagerClassName(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; + } } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java new file mode 100644 index 0000000000000..33f081334b3fb --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java @@ -0,0 +1,67 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Data class containing three components comprising all the data available for the leader broker about other brokers: - + * The local broker data which is written to ZooKeeper by each individual broker (LocalBrokerData). - The time average + * bundle data which is written to ZooKeeper by the leader broker (TimeAverageBrokerData). - The preallocated bundles + * which are not written to ZooKeeper but are maintained by the leader broker (Map). + */ +public class BrokerData { + private LocalBrokerData localData; + private TimeAverageBrokerData timeAverageData; + private Map preallocatedBundleData; + + /** + * Initialize this BrokerData using the most recent local data. + * + * @param localData + * The data local for the broker. + */ + public BrokerData(final LocalBrokerData localData) { + this.localData = localData; + timeAverageData = new TimeAverageBrokerData(); + preallocatedBundleData = new ConcurrentHashMap<>(); + } + + public LocalBrokerData getLocalData() { + return localData; + } + + public void setLocalData(LocalBrokerData localData) { + this.localData = localData; + } + + public TimeAverageBrokerData getTimeAverageData() { + return timeAverageData; + } + + public void setTimeAverageData(TimeAverageBrokerData timeAverageData) { + this.timeAverageData = timeAverageData; + } + + public Map getPreallocatedBundleData() { + return preallocatedBundleData; + } + + public void setPreallocatedBundleData(Map preallocatedBundleData) { + this.preallocatedBundleData = preallocatedBundleData; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java new file mode 100644 index 0000000000000..e07cd017e529e --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java @@ -0,0 +1,93 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; + +/** + * Data class comprising the short term and long term historical data for this bundle. + */ +public class BundleData extends JSONWritable { + // Short term data for this bundle. The time frame of this data is + // determined by the number of short term samples + // and the bundle update period. + private TimeAverageMessageData shortTermData; + + // Long term data for this bundle. The time frame of this data is determined + // by the number of long term samples + // and the bundle update period. + private TimeAverageMessageData longTermData; + + // For JSON only. + public BundleData() { + } + + /** + * Initialize the bundle data. + * + * @param numShortSamples + * Number of short term samples to use. + * @param numLongSamples + * Number of long term samples to use. + */ + public BundleData(final int numShortSamples, final int numLongSamples) { + shortTermData = new TimeAverageMessageData(numShortSamples); + longTermData = new TimeAverageMessageData(numLongSamples); + } + + /** + * Initialize this bundle data and have its histories default to the given stats before the first sample is + * received. + * + * @param numShortSamples + * Number of short term samples to use. + * @param numLongSamples + * Number of long term samples to use. + * @param defaultStats + * The stats to default to before the first sample is received. + */ + public BundleData(final int numShortSamples, final int numLongSamples, final NamespaceBundleStats defaultStats) { + shortTermData = new TimeAverageMessageData(numShortSamples, defaultStats); + longTermData = new TimeAverageMessageData(numLongSamples, defaultStats); + } + + /** + * Update the historical data for this bundle. + * + * @param newSample + * The bundle stats to update this data with. + */ + public void update(final NamespaceBundleStats newSample) { + shortTermData.update(newSample); + longTermData.update(newSample); + } + + public TimeAverageMessageData getShortTermData() { + return shortTermData; + } + + public void setShortTermData(TimeAverageMessageData shortTermData) { + this.shortTermData = shortTermData; + } + + public TimeAverageMessageData getLongTermData() { + return longTermData; + } + + public void setLongTermData(TimeAverageMessageData longTermData) { + this.longTermData = longTermData; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java new file mode 100644 index 0000000000000..3de8a29474f9a --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java @@ -0,0 +1,48 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.yahoo.pulsar.common.util.ObjectMapperFactory; + +/** + * Helper class used to conveniently convert a data class to a JSON. + */ +public class JSONWritable { + + /** + * Get the JSON of this object as a byte[]. + * + * @return A byte[] of this object's JSON. + * @throws JsonProcessingException + */ + @JsonIgnore + public byte[] getJsonBytes() throws JsonProcessingException { + return ObjectMapperFactory.getThreadLocal().writeValueAsBytes(this); + } + + /** + * Get the JSON of this object as a String. + * + * @return A String of this object's JSON. + * @throws JsonProcessingException + */ + @JsonIgnore + public String getJsonString() throws JsonProcessingException { + return ObjectMapperFactory.getThreadLocal().writeValueAsString(this); + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java new file mode 100644 index 0000000000000..2249b2dfa4bf8 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -0,0 +1,345 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; +import com.yahoo.pulsar.common.policies.data.loadbalancer.ResourceUsage; +import com.yahoo.pulsar.common.policies.data.loadbalancer.ServiceLookupData; +import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage; + +/** + * Contains all the data that is maintained locally on each broker. + */ +public class LocalBrokerData extends JSONWritable implements ServiceLookupData { + + // URLs to satisfy contract of ServiceLookupData (used by NamespaceService). + private final String webServiceUrl; + private final String webServiceUrlTls; + private final String pulsarServiceUrl; + private final String pulsarServiceUrlTls; + + // Most recently available system resource usage. + private ResourceUsage cpu; + private ResourceUsage memory; + private ResourceUsage directMemory; + + private ResourceUsage bandwidthIn; + private ResourceUsage bandwidthOut; + + // Message data from the most recent namespace bundle stats. + private double msgThroughputIn; + private double msgThroughputOut; + private double msgRateIn; + private double msgRateOut; + + // Timestamp of last update. + private long lastUpdate; + + // The stats given in the most recent invocation of update. + private Map lastStats; + + private int numTopics; + private int numBundles; + private int numConsumers; + private int numProducers; + + // All bundles belonging to this broker. + private Set bundles; + + // The bundles gained since the last invocation of update. + private Set lastBundleGains; + + // The bundles lost since the last invocation of update. + private Set lastBundleLosses; + + // For JSON only. + public LocalBrokerData() { + this(null, null, null, null); + } + + /** + * Broker data constructor which takes in four URLs to satisfy the contract of ServiceLookupData. + */ + public LocalBrokerData(final String webServiceUrl, final String webServiceUrlTls, final String pulsarServiceUrl, + final String pulsarServiceUrlTls) { + this.webServiceUrl = webServiceUrl; + this.webServiceUrlTls = webServiceUrlTls; + this.pulsarServiceUrl = pulsarServiceUrl; + this.pulsarServiceUrlTls = pulsarServiceUrlTls; + lastStats = new HashMap<>(); + lastUpdate = System.currentTimeMillis(); + cpu = new ResourceUsage(); + memory = new ResourceUsage(); + directMemory = new ResourceUsage(); + bandwidthIn = new ResourceUsage(); + bandwidthOut = new ResourceUsage(); + bundles = new HashSet<>(); + lastBundleGains = new HashSet<>(); + lastBundleLosses = new HashSet<>(); + } + + /** + * Using the system resource usage and bundle stats acquired from the Pulsar client, update this LocalBrokerData. + * + * @param systemResourceUsage + * System resource usage (cpu, memory, and direct memory). + * @param bundleStats + * The bundle stats retrieved from the Pulsar client. + */ + public void update(final SystemResourceUsage systemResourceUsage, + final Map bundleStats) { + updateSystemResourceUsage(systemResourceUsage); + updateBundleData(bundleStats); + lastStats = bundleStats; + lastUpdate = System.currentTimeMillis(); + } + + // Set the cpu, memory, and direct memory to that of the new system resource + // usage data. + private void updateSystemResourceUsage(final SystemResourceUsage systemResourceUsage) { + this.cpu = systemResourceUsage.cpu; + this.memory = systemResourceUsage.memory; + this.directMemory = systemResourceUsage.directMemory; + this.bandwidthIn = systemResourceUsage.bandwidthIn; + this.bandwidthOut = systemResourceUsage.bandwidthOut; + } + + // Aggregate all message, throughput, topic count, bundle count, consumer + // count, and producer count across the + // given data. Also keep track of bundle gains and losses. + private void updateBundleData(final Map bundleStats) { + msgRateIn = 0; + msgRateOut = 0; + msgThroughputIn = 0; + msgThroughputOut = 0; + int totalNumTopics = 0; + int totalNumBundles = 0; + int totalNumConsumers = 0; + int totalNumProducers = 0; + lastBundleGains.clear(); + lastBundleLosses.clear(); + final Iterator oldBundleIterator = bundles.iterator(); + while (oldBundleIterator.hasNext()) { + final String bundle = oldBundleIterator.next(); + if (!bundleStats.containsKey(bundle)) { + // If this bundle is in the old bundle set but not the new one, + // we lost it. + lastBundleLosses.add(bundle); + oldBundleIterator.remove(); + } + } + for (Map.Entry entry : bundleStats.entrySet()) { + final String bundle = entry.getKey(); + final NamespaceBundleStats stats = entry.getValue(); + if (!bundles.contains(bundle)) { + // If this bundle is in the new bundle set but not the old one, + // we gained it. + lastBundleGains.add(bundle); + bundles.add(bundle); + } + msgThroughputIn += stats.msgThroughputIn; + msgThroughputOut += stats.msgThroughputOut; + msgRateIn += stats.msgRateIn; + msgRateOut += stats.msgRateOut; + totalNumTopics += stats.topics; + ++totalNumBundles; + totalNumConsumers += stats.consumerCount; + totalNumProducers += stats.producerCount; + } + numTopics = totalNumTopics; + numBundles = totalNumBundles; + numConsumers = totalNumConsumers; + numProducers = totalNumProducers; + } + + public double getMaxResourceUsage() { + return Math + .max(Math.max(Math.max(cpu.percentUsage(), memory.percentUsage()), + Math.max(directMemory.percentUsage(), bandwidthIn.percentUsage())), bandwidthOut.percentUsage()) + / 100; + } + + public ResourceUsage getCpu() { + return cpu; + } + + public void setCpu(ResourceUsage cpu) { + this.cpu = cpu; + } + + public ResourceUsage getMemory() { + return memory; + } + + public void setMemory(ResourceUsage memory) { + this.memory = memory; + } + + public ResourceUsage getDirectMemory() { + return directMemory; + } + + public void setDirectMemory(ResourceUsage directMemory) { + this.directMemory = directMemory; + } + + public ResourceUsage getBandwidthIn() { + return bandwidthIn; + } + + public void setBandwidthIn(ResourceUsage bandwidthIn) { + this.bandwidthIn = bandwidthIn; + } + + public ResourceUsage getBandwidthOut() { + return bandwidthOut; + } + + public void setBandwidthOut(ResourceUsage bandwidthOut) { + this.bandwidthOut = bandwidthOut; + } + + public Set getLastBundleGains() { + return lastBundleGains; + } + + public void setLastBundleGains(Set lastBundleGains) { + this.lastBundleGains = lastBundleGains; + } + + public Set getLastBundleLosses() { + return lastBundleLosses; + } + + public void setLastBundleLosses(Set lastBundleLosses) { + this.lastBundleLosses = lastBundleLosses; + } + + public long getLastUpdate() { + return lastUpdate; + } + + public void setLastUpdate(long lastUpdate) { + this.lastUpdate = lastUpdate; + } + + public Set getBundles() { + return bundles; + } + + public void setBundles(Set bundles) { + this.bundles = bundles; + } + + public Map getLastStats() { + return lastStats; + } + + public void setLastStats(Map lastStats) { + this.lastStats = lastStats; + } + + public int getNumTopics() { + return numTopics; + } + + public void setNumTopics(int numTopics) { + this.numTopics = numTopics; + } + + public int getNumBundles() { + return numBundles; + } + + public void setNumBundles(int numBundles) { + this.numBundles = numBundles; + } + + public int getNumConsumers() { + return numConsumers; + } + + public void setNumConsumers(int numConsumers) { + this.numConsumers = numConsumers; + } + + public int getNumProducers() { + return numProducers; + } + + public void setNumProducers(int numProducers) { + this.numProducers = numProducers; + } + + public double getMsgThroughputIn() { + return msgThroughputIn; + } + + public void setMsgThroughputIn(double msgThroughputIn) { + this.msgThroughputIn = msgThroughputIn; + } + + public double getMsgThroughputOut() { + return msgThroughputOut; + } + + public void setMsgThroughputOut(double msgThroughputOut) { + this.msgThroughputOut = msgThroughputOut; + } + + public double getMsgRateIn() { + return msgRateIn; + } + + public void setMsgRateIn(double msgRateIn) { + this.msgRateIn = msgRateIn; + } + + public double getMsgRateOut() { + return msgRateOut; + } + + public void setMsgRateOut(double msgRateOut) { + this.msgRateOut = msgRateOut; + } + + @Override + public String getWebServiceUrl() { + return webServiceUrl; + } + + @Override + public String getWebServiceUrlTls() { + return webServiceUrlTls; + } + + @Override + public String getPulsarServiceUrl() { + return pulsarServiceUrl; + } + + @Override + public String getPulsarServiceUrlTls() { + return pulsarServiceUrlTls; + } + +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/PulsarService.java index 218239a9dbf3a..893d5a97cd184 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/PulsarService.java @@ -19,17 +19,23 @@ import java.net.URL; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.util.OrderedSafeExecutor; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.eclipse.jetty.servlet.ServletHolder; import org.slf4j.Logger; @@ -93,7 +99,7 @@ public class PulsarService implements AutoCloseable { private ScheduledFuture loadReportTask = null; private ScheduledFuture loadSheddingTask = null; private ScheduledFuture loadResourceQuotaTask = null; - private LoadManager loadManager = null; + private AtomicReference loadManager = null; private PulsarAdmin adminClient = null; private ZooKeeperClientFactory zkClientFactory = null; private final String bindAddress; @@ -235,7 +241,7 @@ public void start() throws PulsarServerException { this.brokerService = new BrokerService(this); // Start load management service (even if load balancing is disabled) - this.loadManager = new SimpleLoadManagerImpl(this); + this.loadManager = new AtomicReference<>(LoadManager.create(this)); this.startLoadManagementService(); @@ -396,7 +402,7 @@ public Supplier getNamespaceServiceProvider() throws PulsarSer private void startLoadManagementService() throws PulsarServerException { LOG.info("Starting load management service ..."); - this.loadManager.start(); + this.loadManager.get().start(); if (config.isLoadBalancerEnabled()) { LOG.info("Starting load balancer"); @@ -625,7 +631,7 @@ public String getBrokerServiceUrlTls() { return brokerServiceUrlTls; } - public LoadManager getLoadManager() { + public AtomicReference getLoadManager() { return loadManager; } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java new file mode 100644 index 0000000000000..43beb90b8f768 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java @@ -0,0 +1,168 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import java.util.Map; +import java.util.Set; + +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; + +/** + * Data class aggregating the short term and long term data across all bundles belonging to a broker. + */ +public class TimeAverageBrokerData extends JSONWritable { + private double shortTermMsgThroughputIn; + private double shortTermMsgThroughputOut; + private double shortTermMsgRateIn; + private double shortTermMsgRateOut; + private double longTermMsgThroughputIn; + private double longTermMsgThroughputOut; + private double longTermMsgRateIn; + private double longTermMsgRateOut; + + public TimeAverageBrokerData() { + } + + /** + * Initialize a TimeAverageBrokerData. + * + * @param bundles + * The bundles belonging to the broker. + * @param data + * Map from bundle names to the data for that bundle. + * @param defaultStats + * The stats to use when a bundle belonging to this broker is not found in the bundle data map. + */ + public TimeAverageBrokerData(final Set bundles, final Map data, + final NamespaceBundleStats defaultStats) { + reset(bundles, data, defaultStats); + } + + /** + * Reuse this TimeAverageBrokerData using new data. + * + * @param bundles + * The bundles belonging to the broker. + * @param data + * Map from bundle names to the data for that bundle. + * @param defaultStats + * The stats to use when a bundle belonging to this broker is not found in the bundle data map. + */ + public void reset(final Set bundles, final Map data, + final NamespaceBundleStats defaultStats) { + shortTermMsgThroughputIn = 0; + shortTermMsgThroughputOut = 0; + shortTermMsgRateIn = 0; + shortTermMsgRateOut = 0; + + longTermMsgThroughputIn = 0; + longTermMsgThroughputOut = 0; + longTermMsgRateIn = 0; + longTermMsgRateOut = 0; + + for (String bundle : bundles) { + final BundleData bundleData = data.get(bundle); + if (bundleData == null) { + shortTermMsgThroughputIn += defaultStats.msgThroughputIn; + shortTermMsgThroughputOut += defaultStats.msgThroughputOut; + shortTermMsgRateIn += defaultStats.msgRateIn; + shortTermMsgRateOut += defaultStats.msgRateOut; + + longTermMsgThroughputIn += defaultStats.msgThroughputIn; + longTermMsgThroughputOut += defaultStats.msgThroughputOut; + longTermMsgRateIn += defaultStats.msgRateIn; + longTermMsgRateOut += defaultStats.msgRateOut; + } else { + final TimeAverageMessageData shortTermData = bundleData.getShortTermData(); + final TimeAverageMessageData longTermData = bundleData.getLongTermData(); + + shortTermMsgThroughputIn += shortTermData.getMsgThroughputIn(); + shortTermMsgThroughputOut += shortTermData.getMsgThroughputOut(); + shortTermMsgRateIn += shortTermData.getMsgRateIn(); + shortTermMsgRateOut += shortTermData.getMsgRateOut(); + + longTermMsgThroughputIn += longTermData.getMsgThroughputIn(); + longTermMsgThroughputOut += longTermData.getMsgThroughputOut(); + longTermMsgRateIn += longTermData.getMsgRateIn(); + longTermMsgRateOut += longTermData.getMsgRateOut(); + } + } + } + + public double getShortTermMsgThroughputIn() { + return shortTermMsgThroughputIn; + } + + public void setShortTermMsgThroughputIn(double shortTermMsgThroughputIn) { + this.shortTermMsgThroughputIn = shortTermMsgThroughputIn; + } + + public double getShortTermMsgThroughputOut() { + return shortTermMsgThroughputOut; + } + + public void setShortTermMsgThroughputOut(double shortTermMsgThroughputOut) { + this.shortTermMsgThroughputOut = shortTermMsgThroughputOut; + } + + public double getShortTermMsgRateIn() { + return shortTermMsgRateIn; + } + + public void setShortTermMsgRateIn(double shortTermMsgRateIn) { + this.shortTermMsgRateIn = shortTermMsgRateIn; + } + + public double getShortTermMsgRateOut() { + return shortTermMsgRateOut; + } + + public void setShortTermMsgRateOut(double shortTermMsgRateOut) { + this.shortTermMsgRateOut = shortTermMsgRateOut; + } + + public double getLongTermMsgThroughputIn() { + return longTermMsgThroughputIn; + } + + public void setLongTermMsgThroughputIn(double longTermMsgThroughputIn) { + this.longTermMsgThroughputIn = longTermMsgThroughputIn; + } + + public double getLongTermMsgThroughputOut() { + return longTermMsgThroughputOut; + } + + public void setLongTermMsgThroughputOut(double longTermMsgThroughputOut) { + this.longTermMsgThroughputOut = longTermMsgThroughputOut; + } + + public double getLongTermMsgRateIn() { + return longTermMsgRateIn; + } + + public void setLongTermMsgRateIn(double longTermMsgRateIn) { + this.longTermMsgRateIn = longTermMsgRateIn; + } + + public double getLongTermMsgRateOut() { + return longTermMsgRateOut; + } + + public void setLongTermMsgRateOut(double longTermMsgRateOut) { + this.longTermMsgRateOut = longTermMsgRateOut; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java new file mode 100644 index 0000000000000..22cd0fcaab52b --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java @@ -0,0 +1,161 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker; + +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; + +/** + * Data class comprising the average message data over a fixed period of time. + */ +public class TimeAverageMessageData { + // The maximum number of samples this data will consider. + private int maxSamples; + + // The number of samples that are currently available for this data. Always + // at most maxSamples. + private int numSamples; + + // The average throughput-in in bytes per second. + private double msgThroughputIn; + + // The average throughput-out in bytes per second. + private double msgThroughputOut; + + // The average message rate in per second. + private double msgRateIn; + + // The average message rate out per second. + private double msgRateOut; + + // For JSON only. + public TimeAverageMessageData() { + } + + /** + * Initialize this TimeAverageData to 0 values. + * + * @param maxSamples + * The maximum number of samples with which to maintain the average. + */ + public TimeAverageMessageData(final int maxSamples) { + this.maxSamples = maxSamples; + } + + /** + * Initialize this TimeAverageData using default stats. + * + * @param maxSamples + * The maximum number of samples with which to maintain the average. + * @param defaultStats + * The stats to default to. These are overwritten after the first update. + */ + public TimeAverageMessageData(final int maxSamples, final NamespaceBundleStats defaultStats) { + this.maxSamples = maxSamples; + msgThroughputIn = defaultStats.msgThroughputIn; + msgThroughputOut = defaultStats.msgThroughputOut; + msgRateIn = defaultStats.msgRateIn; + msgRateOut = defaultStats.msgRateOut; + } + + /** + * Update using new samples for the message data. + * + * @param newMsgThroughputIn + * Most recently observed throughput in. + * @param newMsgThroughputOut + * Most recently observed throughput out. + * @param newMsgRateIn + * Most recently observed message rate in. + * @param newMsgRateOut + * Most recently observed message rate out. + */ + public void update(final double newMsgThroughputIn, final double newMsgThroughputOut, final double newMsgRateIn, + final double newMsgRateOut) { + // If max samples has been reached, don't increase numSamples. + numSamples = Math.min(numSamples + 1, maxSamples); + msgThroughputIn = getUpdatedValue(msgThroughputIn, newMsgThroughputIn); + msgThroughputOut = getUpdatedValue(msgThroughputOut, newMsgThroughputOut); + msgRateIn = getUpdatedValue(msgRateIn, newMsgRateIn); + msgRateOut = getUpdatedValue(msgRateOut, newMsgRateOut); + } + + /** + * Update using a new bundle sample. + * + * @param newSample + * Most recently observed bundle stats. + */ + public void update(final NamespaceBundleStats newSample) { + update(newSample.msgThroughputIn, newSample.msgThroughputOut, newSample.msgRateIn, newSample.msgRateOut); + } + + // Update the average of a sample using the number of samples, the previous + // average, and a new sample. + private double getUpdatedValue(final double oldAverage, final double newSample) { + // Note that for numSamples == 1, this returns newSample. + // This ensures that default stats get overwritten after the first + // update. + return ((numSamples - 1) * oldAverage + newSample) / numSamples; + } + + public int getMaxSamples() { + return maxSamples; + } + + public void setMaxSamples(int maxSamples) { + this.maxSamples = maxSamples; + } + + public int getNumSamples() { + return numSamples; + } + + public void setNumSamples(int numSamples) { + this.numSamples = numSamples; + } + + public double getMsgThroughputIn() { + return msgThroughputIn; + } + + public void setMsgThroughputIn(double msgThroughputIn) { + this.msgThroughputIn = msgThroughputIn; + } + + public double getMsgThroughputOut() { + return msgThroughputOut; + } + + public void setMsgThroughputOut(double msgThroughputOut) { + this.msgThroughputOut = msgThroughputOut; + } + + public double getMsgRateIn() { + return msgRateIn; + } + + public void setMsgRateIn(double msgRateIn) { + this.msgRateIn = msgRateIn; + } + + public double getMsgRateOut() { + return msgRateOut; + } + + public void setMsgRateOut(double msgRateOut) { + this.msgRateOut = msgRateOut; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/BrokerStats.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/BrokerStats.java index c9341749953fa..056d722a24295 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/BrokerStats.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/BrokerStats.java @@ -17,6 +17,7 @@ import java.io.OutputStream; import java.util.Collection; +import java.util.Collections; import java.util.Map; import javax.ws.rs.GET; @@ -28,6 +29,7 @@ import javax.ws.rs.core.Response.Status; import javax.ws.rs.core.StreamingOutput; +import com.yahoo.pulsar.broker.loadbalance.LoadManager; import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -152,7 +154,7 @@ public LoadReport getLoadReport() throws Exception { // Ensure super user access only validateSuperUserAccess(); try { - return ((SimpleLoadManagerImpl) pulsar().getLoadManager()).generateLoadReport(); + return (pulsar().getLoadManager().get()).generateLoadReport(); } catch (Exception e) { log.error("[{}] Failed to generate LoadReport for broker, reason [{}]", clientAppId(), e.getMessage(), e); throw new RestException(e); @@ -169,8 +171,12 @@ public Map> getBrokerResourceAvailability(@PathPa @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) throws Exception { try { NamespaceName ns = new NamespaceName(property, cluster, namespace); - SimpleLoadManagerImpl lm = (SimpleLoadManagerImpl) (pulsar().getLoadManager()); - return lm.getResourceAvailabilityFor(ns).asMap(); + LoadManager lm = pulsar().getLoadManager().get(); + if (lm instanceof SimpleLoadManagerImpl) { + return ((SimpleLoadManagerImpl) lm).getResourceAvailabilityFor(ns).asMap(); + } else { + return null; + } } catch (Exception e) { log.error("Unable to get Resource Availability - [{}]", e); throw new RestException(e); diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/Brokers.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/Brokers.java index 3fea37d54bc4b..8557c0944077f 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/Brokers.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/admin/Brokers.java @@ -29,6 +29,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response.Status; +import com.yahoo.pulsar.broker.loadbalance.LoadManager; import org.apache.bookkeeper.util.ZkUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; @@ -68,7 +69,7 @@ public Set getActiveBrokers(@PathParam("cluster") String cluster) throws try { // Add Native brokers - return pulsar().getLocalZkCache().getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT); + return pulsar().getLocalZkCache().getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT); } catch (Exception e) { LOG.error(String.format("[%s] Failed to get active broker list: cluster=%s", clientAppId(), cluster), e); throw new RestException(e); diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java new file mode 100644 index 0000000000000..b2fc77450945f --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java @@ -0,0 +1,45 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance; + +import java.util.Set; + +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.ServiceConfiguration; + +/** + * Load management component which determines what brokers should not be considered for topic placement by the placement + * strategy. For example, the placement strategy may determine that the broker with the least msg/s should get the + * bundle assignment, but we may not want to consider brokers whose CPU usage is very high. Thus, we could use a filter + * to blacklist brokers with high CPU usage. + */ +public interface BrokerFilter { + + /** + * From the given set of available broker candidates, filter those using the load data. + * + * @param brokers + * The currently available brokers that have not already been filtered. This set may be modified by + * filter. + * @param bundleToAssign + * The data for the bundle to assign. + * @param loadData + * The load data from the leader broker. + * @param conf + * The service configuration. + */ + void filter(Set brokers, BundleData bundleToAssign, LoadData loadData, ServiceConfiguration conf); +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadData.java new file mode 100644 index 0000000000000..f78dcd98b09e2 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadData.java @@ -0,0 +1,53 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import com.yahoo.pulsar.broker.BrokerData; +import com.yahoo.pulsar.broker.BundleData; + +/** + * This class represents all data that could be relevant when making a load management decision. + */ +public class LoadData { + /** + * Map from broker names to their available data. + */ + private final Map brokerData; + + /** + * Map from bundle names to their time-sensitive aggregated data. + */ + private final Map bundleData; + + /** + * Initialize a LoadData. + */ + public LoadData() { + this.brokerData = new ConcurrentHashMap<>(); + this.bundleData = new ConcurrentHashMap<>(); + } + + public Map getBrokerData() { + return brokerData; + } + + public Map getBundleData() { + return bundleData; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadManager.java index ce6ef5666c782..2cadbef2b6835 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadManager.java @@ -18,9 +18,15 @@ import java.util.List; import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.broker.PulsarService; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; +import com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import com.yahoo.pulsar.broker.stats.Metrics; import com.yahoo.pulsar.common.naming.ServiceUnitId; import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * LoadManager runs though set of load reports collected from different brokers and generates a recommendation of @@ -30,6 +36,9 @@ * Concrete Load Manager is also return the least loaded broker that should own the new namespace. */ public interface LoadManager { + Logger log = LoggerFactory.getLogger(LoadManager.class); + + String LOADBALANCE_BROKERS_ROOT = "/loadbalance/brokers"; public void start() throws PulsarServerException; @@ -87,4 +96,34 @@ public interface LoadManager { public void disableBroker() throws Exception; public void stop() throws PulsarServerException; + + /** + * Initialize this LoadManager. + * + * @param pulsar + * The service to initialize this with. + */ + public void initialize(PulsarService pulsar); + + static LoadManager create(final PulsarService pulsar) { + try { + final ServiceConfiguration conf = pulsar.getConfiguration(); + final Class loadManagerClass = Class.forName(conf.getLoadManagerClassName()); + // Assume there is a constructor with one argument of PulsarService. + final Object loadManagerInstance = loadManagerClass.newInstance(); + if (loadManagerInstance instanceof LoadManager) { + final LoadManager casted = (LoadManager) loadManagerInstance; + casted.initialize(pulsar); + return casted; + } else if (loadManagerInstance instanceof ModularLoadManager) { + final LoadManager casted = new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance); + casted.initialize(pulsar); + return casted; + } + } catch (Exception e) { + log.warn("Error when trying to create load manager: {}"); + } + // If we failed to create a load manager, default to SimpleLoadManagerImpl. + return new SimpleLoadManagerImpl(pulsar); + } } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadReportUpdaterTask.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadReportUpdaterTask.java index 514cb2afe43a8..d2c5f90ebd3dd 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadReportUpdaterTask.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadReportUpdaterTask.java @@ -16,6 +16,7 @@ package com.yahoo.pulsar.broker.loadbalance; import java.lang.Runnable; +import java.util.concurrent.atomic.AtomicReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,16 +26,16 @@ */ public class LoadReportUpdaterTask implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(LoadReportUpdaterTask.class); - private final LoadManager loadManager; + private final AtomicReference loadManager; - public LoadReportUpdaterTask(LoadManager manager) { + public LoadReportUpdaterTask(AtomicReference manager) { loadManager = manager; } @Override public void run() { try { - loadManager.writeLoadReportOnZookeeper(); + loadManager.get().writeLoadReportOnZookeeper(); } catch (Exception e) { LOG.warn("Unable to write load report on Zookeeper - [{}]", e); } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadResourceQuotaUpdaterTask.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadResourceQuotaUpdaterTask.java index 34b61a25e6de7..9a6a683a1ec7d 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadResourceQuotaUpdaterTask.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadResourceQuotaUpdaterTask.java @@ -19,22 +19,23 @@ import org.slf4j.LoggerFactory; import java.lang.Runnable; +import java.util.concurrent.atomic.AtomicReference; /** * LoadManager namespace bundle quota update task */ public class LoadResourceQuotaUpdaterTask implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(LoadResourceQuotaUpdaterTask.class); - private final LoadManager loadManager; + private final AtomicReference loadManager; - public LoadResourceQuotaUpdaterTask(LoadManager loadManager) { + public LoadResourceQuotaUpdaterTask(AtomicReference loadManager) { this.loadManager = loadManager; } @Override public void run() { try { - this.loadManager.writeResourceQuotasToZooKeeper(); + this.loadManager.get().writeResourceQuotasToZooKeeper(); } catch (Exception e) { LOG.warn("Error write resource quota", e); } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java new file mode 100644 index 0000000000000..f9835087b7a6c --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java @@ -0,0 +1,37 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance; + +import java.util.Map; + +import com.yahoo.pulsar.broker.ServiceConfiguration; + +/** + * Load management component which determines the criteria for unloading bundles. + */ +public interface LoadSheddingStrategy { + + /** + * Recommend that all of the returned bundles be unloaded. + * + * @param loadData + * The load data to used to make the unloading decision. + * @param conf + * The service configuration. + * @return A map from all selected bundles to the brokers on which they reside. + */ + Map findBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingTask.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingTask.java index 44248f0a43e6f..9dfa540945a3d 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingTask.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingTask.java @@ -19,22 +19,23 @@ import org.slf4j.LoggerFactory; import java.lang.Runnable; +import java.util.concurrent.atomic.AtomicReference; /** * LoadManager load shedding task */ public class LoadSheddingTask implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(LoadSheddingTask.class); - private final LoadManager loadManager; + private final AtomicReference loadManager; - public LoadSheddingTask(LoadManager loadManager) { + public LoadSheddingTask(AtomicReference loadManager) { this.loadManager = loadManager; } @Override public void run() { try { - loadManager.doLoadShedding(); + loadManager.get().doLoadShedding(); } catch (Exception e) { LOG.warn("Error during the load shedding", e); } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java new file mode 100644 index 0000000000000..14268d5ec9fc0 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java @@ -0,0 +1,91 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.broker.PulsarService; +import com.yahoo.pulsar.common.naming.ServiceUnitId; + +/** + * New proposal for a load manager interface which attempts to use more intuitive method names and provide a starting + * place for new load manager proposals. + */ +public interface ModularLoadManager { + + /** + * As any broker, disable the broker this manager is running on. + * + * @throws PulsarServerException + * If ZooKeeper failed to disable the broker. + */ + void disableBroker() throws PulsarServerException; + + /** + * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new + * brokers. + */ + void doLoadShedding(); + + /** + * As the leader broker, attempt to automatically detect and split hot namespace bundles. + */ + void doNamespaceBundleSplit(); + + /** + * Initialize this load manager using the given pulsar service. + */ + void initialize(PulsarService pulsar); + + /** + * As the leader broker, find a suitable broker for the assignment of the given bundle. + * + * @param serviceUnit + * ServiceUnitId for the bundle. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + String selectBrokerForAssignment(ServiceUnitId serviceUnit); + + /** + * As any broker, start the load manager. + * + * @throws PulsarServerException + * If an unexpected error prevented the load manager from being started. + */ + void start() throws PulsarServerException; + + /** + * As any broker, stop the load manager. + * + * @throws PulsarServerException + * If an unexpected error occurred when attempting to stop the load manager. + */ + void stop() throws PulsarServerException; + + /** + * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. + */ + void updateLocalBrokerData(); + + /** + * As any broker, write the local broker data to ZooKeeper. + */ + void writeBrokerDataOnZooKeeper(); + + /** + * As the leader broker, write bundle data aggregated from all brokers to ZooKeeper. + */ + void writeBundleDataOnZooKeeper(); +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java new file mode 100644 index 0000000000000..694e2593fa809 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java @@ -0,0 +1,62 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance; + +import java.util.Set; + +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; + +/** + * Interface which serves as a component for ModularLoadManagerImpl, flexibly allowing the injection of potentially + * complex strategies. + */ +public interface ModularLoadManagerStrategy { + + /** + * Find a suitable broker to assign the given bundle to. + * + * @param candidates + * The candidates for which the bundle may be assigned. + * @param bundleToAssign + * The data for the bundle to assign. + * @param loadData + * The load data from the leader broker. + * @param conf + * The service configuration. + * @return The name of the selected broker as it appears on ZooKeeper. + */ + String selectBroker(Set candidates, BundleData bundleToAssign, LoadData loadData, + ServiceConfiguration conf); + + /** + * Create a placement strategy using the configuration. + * + * @param conf + * ServiceConfiguration to use. + * @return A placement strategy from the given configurations. + */ + static ModularLoadManagerStrategy create(final ServiceConfiguration conf) { + try { + // Only one strategy at the moment. + return new LeastLongTermMessageRate(conf); + } catch (Exception e) { + // Ignore + } + return new LeastLongTermMessageRate(conf); + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/DeviationShedder.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/DeviationShedder.java new file mode 100644 index 0000000000000..7906b7040fe0c --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/DeviationShedder.java @@ -0,0 +1,151 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import java.util.HashMap; +import java.util.Map; +import java.util.TreeSet; + +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import com.yahoo.pulsar.broker.BrokerData; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.loadbalance.LoadData; +import com.yahoo.pulsar.broker.loadbalance.LoadSheddingStrategy; + +/** + * An abstract class which makes a LoadSheddingStrategy which makes decisions based on standard deviation easier to + * implement. Assuming there exists some real number metric which may estimate the load on a server, this load shedding + * strategy calculates the standard deviation with respect to that metric and sheds load on brokers whose standard + * deviation is above some threshold. + */ +public abstract class DeviationShedder implements LoadSheddingStrategy { + // A Set of pairs is used in favor of a Multimap for simplicity. + protected TreeSet> metricTreeSetCache; + protected TreeSet> bundleTreeSetCache; + + /** + * Initialize this DeviationShedder. + */ + public DeviationShedder() { + bundleTreeSetCache = new TreeSet<>(); + metricTreeSetCache = new TreeSet<>(); + } + + // Measure the load incurred by a bundle. + protected abstract double bundleValue(String bundle, BrokerData brokerData, ServiceConfiguration conf); + + // Measure the load suffered by a broker. + protected abstract double brokerValue(BrokerData brokerData, ServiceConfiguration conf); + + // Get the threshold above which the standard deviation of a broker is large + // enough to warrant unloading bundles. + protected abstract double getDeviationThreshold(ServiceConfiguration conf); + + /** + * Recommend that all of the returned bundles be unloaded based on observing excessive standard deviations according + * to some metric. + * + * @param loadData + * The load data to used to make the unloading decision. + * @param conf + * The service configuration. + * @return A map from all selected bundles to the brokers on which they reside. + */ + @Override + public Map findBundlesForUnloading(final LoadData loadData, final ServiceConfiguration conf) { + final Map result = new HashMap<>(); + bundleTreeSetCache.clear(); + metricTreeSetCache.clear(); + double sum = 0; + double squareSum = 0; + final Map brokerDataMap = loadData.getBrokerData(); + + // Treating each broker as a data point, calculate the sum and squared + // sum of the evaluated broker metrics. + // These may be used to calculate the standard deviation. + for (Map.Entry entry : brokerDataMap.entrySet()) { + final double value = brokerValue(entry.getValue(), conf); + sum += value; + squareSum += value * value; + metricTreeSetCache.add(new ImmutablePair<>(value, entry.getKey())); + } + // Mean cannot change by just moving around bundles. + final double mean = sum / brokerDataMap.size(); + double standardDeviation = Math.sqrt(squareSum / brokerDataMap.size() - mean * mean); + final double deviationThreshold = getDeviationThreshold(conf); + String lastMostOverloaded = null; + // While the most loaded broker is above the standard deviation + // threshold, continue to move bundles. + while ((metricTreeSetCache.last().getKey() - mean) / standardDeviation > deviationThreshold) { + final Pair mostLoadedPair = metricTreeSetCache.last(); + final double highestValue = mostLoadedPair.getKey(); + final String mostLoaded = mostLoadedPair.getValue(); + + final Pair leastLoadedPair = metricTreeSetCache.first(); + final double leastValue = leastLoadedPair.getKey(); + final String leastLoaded = metricTreeSetCache.first().getValue(); + + if (!mostLoaded.equals(lastMostOverloaded)) { + // Reset the bundle tree set now that a different broker is + // being considered. + bundleTreeSetCache.clear(); + for (String bundle : brokerDataMap.get(mostLoaded).getLocalData().getBundles()) { + if (!result.containsKey(bundle)) { + // Don't consider bundles that are already going to be + // moved. + bundleTreeSetCache.add( + new ImmutablePair<>(bundleValue(bundle, brokerDataMap.get(mostLoaded), conf), bundle)); + } + } + lastMostOverloaded = mostLoaded; + } + boolean selected = false; + while (!(bundleTreeSetCache.isEmpty() || selected)) { + Pair mostExpensivePair = bundleTreeSetCache.pollLast(); + double loadIncurred = mostExpensivePair.getKey(); + // When the bundle is moved, we want the now least loaded server + // to have lower overall load than the + // most loaded server does not. Thus, we will only consider + // moving the bundle if this condition + // holds, and otherwise we will try the next bundle. + if (loadIncurred + leastValue < highestValue) { + // Update the standard deviation and replace the old load + // values in the broker tree set with the + // load values assuming this move took place. + final String bundleToMove = mostExpensivePair.getValue(); + result.put(bundleToMove, mostLoaded); + metricTreeSetCache.remove(mostLoadedPair); + metricTreeSetCache.remove(leastLoadedPair); + final double newHighLoad = highestValue - loadIncurred; + final double newLowLoad = leastValue - loadIncurred; + squareSum -= highestValue * highestValue + leastValue * leastValue; + squareSum += newHighLoad * newHighLoad + newLowLoad * newLowLoad; + standardDeviation = Math.sqrt(squareSum / brokerDataMap.size() - mean * mean); + metricTreeSetCache.add(new ImmutablePair<>(newLowLoad, leastLoaded)); + metricTreeSetCache.add(new ImmutablePair<>(newHighLoad, mostLoaded)); + selected = true; + } + } + if (!selected) { + // Move on to the next broker if no bundle could be moved. + metricTreeSetCache.pollLast(); + } + } + return result; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java new file mode 100644 index 0000000000000..8766c148a0406 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java @@ -0,0 +1,118 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import java.util.ArrayList; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.yahoo.pulsar.broker.BrokerData; +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.TimeAverageBrokerData; +import com.yahoo.pulsar.broker.TimeAverageMessageData; +import com.yahoo.pulsar.broker.loadbalance.LoadData; +import com.yahoo.pulsar.broker.loadbalance.ModularLoadManagerStrategy; + +/** + * Placement strategy which selects a broker based on which one has the least long term message rate. + */ +public class LeastLongTermMessageRate implements ModularLoadManagerStrategy { + private static Logger log = LoggerFactory.getLogger(LeastLongTermMessageRate.class); + + // Maintain this list to reduce object creation. + private ArrayList bestBrokers; + + public LeastLongTermMessageRate(final ServiceConfiguration conf) { + bestBrokers = new ArrayList<>(); + } + + // Form a score for a broker using its preallocated bundle data and time average data. + // This is done by summing all preallocated long-term message rates and adding them to the broker's overall + // long-term message rate, which is itself the sum of the long-term message rate of every allocated bundle. + // Once the total long-term message rate is calculated, the score is then weighted by + // max_usage < overload_threshold ? 1 / (overload_threshold - max_usage): Inf + // This weight attempts to discourage the placement of bundles on brokers whose system resource usage is high. + private static double getScore(final BrokerData brokerData, final ServiceConfiguration conf) { + final double overloadThreshold = conf.getLoadBalancerBrokerOverloadedThresholdPercentage() / 100; + double totalMessageRate = 0; + for (BundleData bundleData : brokerData.getPreallocatedBundleData().values()) { + final TimeAverageMessageData longTermData = bundleData.getLongTermData(); + totalMessageRate += longTermData.getMsgRateIn() + longTermData.getMsgRateOut(); + } + final TimeAverageBrokerData timeAverageData = brokerData.getTimeAverageData(); + final double maxUsage = brokerData.getLocalData().getMaxResourceUsage(); + if (maxUsage > overloadThreshold) { + return Double.POSITIVE_INFINITY; + } + // 1 / weight is the proportion of load this machine should receive in + // proportion to a machine with no system resource burden. + // This attempts to spread out the load in such a way that + // machines only become overloaded if there is too much + // load for the system to handle (e.g., all machines are + // at least nearly overloaded). + final double weight = maxUsage < overloadThreshold ? 1 / (overloadThreshold - maxUsage) + : Double.POSITIVE_INFINITY; + final double totalMessageRateEstimate = totalMessageRate + timeAverageData.getLongTermMsgRateIn() + + timeAverageData.getLongTermMsgRateOut(); + return weight * totalMessageRateEstimate; + } + + /** + * Find a suitable broker to assign the given bundle to. + * + * @param candidates + * The candidates for which the bundle may be assigned. + * @param bundleToAssign + * The data for the bundle to assign. + * @param loadData + * The load data from the leader broker. + * @param conf + * The service configuration. + * @return The name of the selected broker as it appears on ZooKeeper. + */ + @Override + public String selectBroker(final Set candidates, final BundleData bundleToAssign, final LoadData loadData, + final ServiceConfiguration conf) { + bestBrokers.clear(); + double minScore = Double.POSITIVE_INFINITY; + // Maintain of list of all the best scoring brokers and then randomly + // select one of them at the end. + for (String broker : candidates) { + final double score = getScore(loadData.getBrokerData().get(broker), conf); + log.info("{} got score {}", broker, score); + if (score < minScore) { + // Clear best brokers since this score beats the other brokers. + bestBrokers.clear(); + bestBrokers.add(broker); + minScore = score; + } else if (score == minScore) { + // Add this broker to best brokers since it ties with the best + // score. + bestBrokers.add(broker); + } + } + if (bestBrokers.isEmpty()) { + // All brokers are overloaded. + // Assign randomly in this case. + bestBrokers.addAll(candidates); + } + return bestBrokers.get(ThreadLocalRandom.current().nextInt(bestBrokers.size())); + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java index 31d6e06d6a19d..8199abfd05c4b 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java @@ -57,7 +57,8 @@ public LinuxBrokerHostUsageImpl(PulsarService pulsar) { this.systemBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); this.lastCollection = 0L; this.usage = new SystemResourceUsage(); - pulsar.getLoadManagerExecutor().scheduleAtFixedRate(this::calculateBrokerHostUsage, 0, hostUsageCheckIntervalMin, TimeUnit.MINUTES); + pulsar.getLoadManagerExecutor().scheduleAtFixedRate(this::calculateBrokerHostUsage, 0, + hostUsageCheckIntervalMin, TimeUnit.MINUTES); } @Override @@ -72,6 +73,7 @@ private void calculateBrokerHostUsage() { double totalNicUsageRx = getTotalNicUsageRxKb(nics); double totalCpuLimit = getTotalCpuLimit(); CpuStat cpuStat = getTotalCpuUsage(); + SystemResourceUsage usage = new SystemResourceUsage(); long now = System.currentTimeMillis(); @@ -111,25 +113,20 @@ private double getTotalCpuLimit() { /** * Reads first line of /proc/stat to get total cpu usage. + * *
      *     cpu  user   nice system idle    iowait irq softirq steal guest guest_nice
      *     cpu  317808 128  58637  2503692 7634   0   13472   0     0     0
      * 
- * 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 substracting the idle cycles, - * this would include iowait, irq and steal. + * + * 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 substracting the idle cycles, this would include iowait, irq and steal. */ private CpuStat getTotalCpuUsage() { try (Stream stream = Files.lines(Paths.get("/proc/stat"))) { - String[] words = stream - .findFirst() - .get().split("\\s+"); + String[] words = stream.findFirst().get().split("\\s+"); - long total = Arrays.stream(words) - .filter(s -> !s.contains("cpu")) - .mapToLong(Long::parseLong) - .sum(); + long total = Arrays.stream(words).filter(s -> !s.contains("cpu")).mapToLong(Long::parseLong).sum(); long idle = Long.parseLong(words[4]); @@ -148,9 +145,7 @@ private ResourceUsage getMemUsage() { private List getNics() { try (Stream stream = Files.list(Paths.get("/sys/class/net/"))) { - return stream - .filter(this::isPhysicalNic) - .map(path -> path.getFileName().toString()) + return stream.filter(this::isPhysicalNic).map(path -> path.getFileName().toString()) .collect(Collectors.toList()); } catch (IOException e) { LOG.error("Failed to find NICs", e); @@ -159,22 +154,16 @@ private List getNics() { } private boolean isPhysicalNic(Path path) { - try { - path = Files.isSymbolicLink(path) ? Files.readSymbolicLink(path) : path; - if (!path.toString().contains("/virtual/")) { - try { - Files.readAllBytes(path.resolve("speed")); - return true; - } catch (Exception e) { - // wireless nics don't report speed, ignore them. - return false; - } + if (!path.toString().contains("/virtual/")) { + try { + Files.readAllBytes(path.resolve("speed")); + return true; + } catch (Exception e) { + // wireless nics don't report speed, ignore them. + return false; } - return false; - } catch (IOException e) { - LOG.error("Failed to read link target for NIC " + path, e); - return false; } + return false; } private Path getNicSpeedPath(String nic) { diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LoadManagerShared.java new file mode 100644 index 0000000000000..ee99cba545eef --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -0,0 +1,158 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import com.yahoo.pulsar.broker.loadbalance.BrokerHostUsage; +import com.yahoo.pulsar.common.naming.NamespaceName; +import com.yahoo.pulsar.common.naming.ServiceUnitId; +import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.HashSet; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * This class contains code which in shared between the two load manager implementations. + */ +public class LoadManagerShared { + public static final Logger log = LoggerFactory.getLogger(LoadManagerShared.class); + + // Value of prefix "mibi" (e.g., number of bytes in a mibibyte). + public static final int MIBI = 1024 * 1024; + + // Cache for primary brokers according to policies. + private static final Set primariesCache = new HashSet<>(); + + // Cache for shard brokers according to policies. + private static final Set sharedCache = new HashSet<>(); + + // Don't allow construction: static method namespace only. + private LoadManagerShared() { + } + + // Determines the brokers available for the given service unit according to the given policies. + // The brokers are put into brokerCandidateCache. + public static synchronized void applyPolicies(final ServiceUnitId serviceUnit, + final SimpleResourceAllocationPolicies policies, final Set brokerCandidateCache, + final Set availableBrokers) { + primariesCache.clear(); + sharedCache.clear(); + NamespaceName namespace = serviceUnit.getNamespaceObject(); + boolean isIsolationPoliciesPresent = policies.IsIsolationPoliciesPresent(namespace); + 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; + try { + brokerUrl = new URL(brokerUrlString); + } catch (MalformedURLException e) { + log.error("Unable to parse brokerUrl from ResourceUnitId - [{}]", 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 (log.isDebugEnabled()) { + log.debug("Added Primary Broker - [{}] as possible Candidates for" + + " namespace - [{}] with policies", brokerUrl.getHost(), namespace.toString()); + } + } else if (policies.isSharedBroker(brokerUrl.getHost())) { + sharedCache.add(broker); + if (log.isDebugEnabled()) { + log.debug( + "Added Shared Broker - [{}] as possible " + + "Candidates for namespace - [{}] with policies", + brokerUrl.getHost(), namespace.toString()); + } + } else { + if (log.isDebugEnabled()) { + log.debug("Skipping Broker - [{}] not primary broker and not shared" + " for namespace - [{}] ", + brokerUrl.getHost(), namespace.toString()); + } + + } + } else { + if (policies.isSharedBroker(brokerUrl.getHost())) { + sharedCache.add(broker); + log.debug("Added Shared Broker - [{}] as possible Candidates for namespace - [{}]", + brokerUrl.getHost(), namespace.toString()); + } + } + } + if (isIsolationPoliciesPresent) { + brokerCandidateCache.addAll(primariesCache); + if (policies.shouldFailoverToSecondaries(namespace, primariesCache.size())) { + log.debug( + "Not enough of primaries [{}] available for namespace - [{}], " + + "adding shared [{}] as possible candidate owners", + primariesCache.size(), namespace.toString(), sharedCache.size()); + brokerCandidateCache.addAll(sharedCache); + } + } else { + log.debug( + "Policies not present for namespace - [{}] so only " + + "considering shared [{}] brokers for possible owner", + namespace.toString(), sharedCache.size()); + brokerCandidateCache.addAll(sharedCache); + } + } + + // From a full bundle name, extract the bundle range. + public static String getBundleRangeFromBundleName(String bundleName) { + // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF + int pos = bundleName.lastIndexOf("/"); + checkArgument(pos != -1); + return bundleName.substring(pos + 1, bundleName.length()); + } + + // From a full bundle name, extract the namespace name. + public static String getNamespaceNameFromBundleName(String bundleName) { + // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF + int pos = bundleName.lastIndexOf("/"); + checkArgument(pos != -1); + return bundleName.substring(0, pos); + } + + // Get the system resource usage for this broker. + public static SystemResourceUsage getSystemResourceUsage(final BrokerHostUsage brokerHostUsage) throws IOException { + SystemResourceUsage systemResourceUsage = brokerHostUsage.getBrokerHostUsage(); + + // Override System memory usage and limit with JVM heap usage and limit + long maxHeapMemoryInBytes = Runtime.getRuntime().maxMemory(); + long memoryUsageInBytes = Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + systemResourceUsage.memory.usage = (double) memoryUsageInBytes / MIBI; + systemResourceUsage.memory.limit = (double) maxHeapMemoryInBytes / MIBI; + + // Collect JVM direct memory + systemResourceUsage.directMemory.usage = (double) (sun.misc.SharedSecrets.getJavaNioAccess() + .getDirectBufferPool().getMemoryUsed() / MIBI); + systemResourceUsage.directMemory.limit = (double) (sun.misc.VM.maxDirectMemory() / MIBI); + + return systemResourceUsage; + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java new file mode 100644 index 0000000000000..d4ef9e8696e29 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -0,0 +1,593 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import com.yahoo.pulsar.broker.loadbalance.LoadManager; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.SystemUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; +import com.yahoo.pulsar.broker.BrokerData; +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.LocalBrokerData; +import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.broker.PulsarService; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.TimeAverageBrokerData; +import com.yahoo.pulsar.broker.loadbalance.BrokerFilter; +import com.yahoo.pulsar.broker.loadbalance.BrokerHostUsage; +import com.yahoo.pulsar.broker.loadbalance.LoadData; +import com.yahoo.pulsar.broker.loadbalance.LoadSheddingStrategy; +import com.yahoo.pulsar.broker.loadbalance.ModularLoadManager; +import com.yahoo.pulsar.broker.loadbalance.ModularLoadManagerStrategy; +import com.yahoo.pulsar.client.admin.PulsarAdmin; +import com.yahoo.pulsar.common.naming.ServiceUnitId; +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; +import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage; +import com.yahoo.pulsar.common.util.ObjectMapperFactory; +import com.yahoo.pulsar.zookeeper.ZooKeeperCacheListener; +import com.yahoo.pulsar.zookeeper.ZooKeeperChildrenCache; +import com.yahoo.pulsar.zookeeper.ZooKeeperDataCache; + +public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCacheListener { + private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); + + // Path to ZNode whose children contain BundleData jsons for each bundle (new API version of ResourceQuota). + public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; + + // Default message rate to assume for unseen bundles. + public static final double DEFAULT_MESSAGE_RATE = 50; + + // Default message throughput to assume for unseen bundles. + // Note that the default message size is implicitly defined as DEFAULT_MESSAGE_THROUGHPUT / DEFAULT_MESSAGE_RATE. + public static final double DEFAULT_MESSAGE_THROUGHPUT = 50000; + + // The number of effective samples to keep for observing long term data. + public static final int NUM_LONG_SAMPLES = 1000; + + // The number of effective samples to keep for observing short term data. + public static final int NUM_SHORT_SAMPLES = 10; + + // Path to ZNode containing TimeAverageBrokerData jsons for each broker. + public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; + + // Cache of PulsarAdmins for each broker. + private LoadingCache adminCache; + + // ZooKeeper Cache of the currently available active brokers. + // availableActiveBrokers.get() will return a set of the broker names without an http prefix. + private ZooKeeperChildrenCache availableActiveBrokers; + + // Set of broker candidates to reuse so that object creation is avoided. + private final Set brokerCandidateCache; + + // ZooKeeper cache of the local broker data, stored in LoadManager.LOADBALANCE_BROKER_ROOT. + private ZooKeeperDataCache brokerDataCache; + + // Broker host usage object used to calculate system resource usage. + private BrokerHostUsage brokerHostUsage; + + // Path to the ZNode containing the LocalBrokerData json for this broker. + private String brokerZnodePath; + + // Service configuration belonging to the pulsar service. + private ServiceConfiguration conf; + + // The default bundle stats which are used to initialize historic data. + // This data is overriden after the bundle receives its first sample. + private final NamespaceBundleStats defaultStats; + + // Used to filter brokers from being selected for assignment. + private final List filterPipeline; + + // Timestamp of last invocation of updateBundleData. + private long lastBundleDataUpdate; + + // Pipeline used to determine what namespaces, if any, should be unloaded. + private final List loadSheddingPipeline; + + // Local data for the broker this is running on. + private LocalBrokerData localData; + + // Load data comprising data available for each broker. + private final LoadData loadData; + + // Used to determine whether a bundle is preallocated. + private final Map preallocatedBundleToBroker; + + // Strategy used to determine where new topics should be placed. + private ModularLoadManagerStrategy placementStrategy; + + // Policies used to determine which brokers are available for particular namespaces. + private SimpleResourceAllocationPolicies policies; + + // Pulsar service used to initialize this. + private PulsarService pulsar; + + // Cache for primary brokers according to policies. + private final Set primariesCache; + + // Executor service used to regularly update broker data. + private final ScheduledExecutorService scheduler; + + // Cache for shard brokers according to policies. + private final Set sharedCache; + + // ZooKeeper belonging to the pulsar service. + private ZooKeeper zkClient; + + /** + * Initializes fields which do not depend on PulsarService. initialize(PulsarService) should subsequently be called. + */ + public ModularLoadManagerImpl() { + brokerCandidateCache = new HashSet<>(); + defaultStats = new NamespaceBundleStats(); + filterPipeline = new ArrayList<>(); + loadData = new LoadData(); + loadSheddingPipeline = new ArrayList<>(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + primariesCache = new HashSet<>(); + scheduler = Executors.newScheduledThreadPool(1); + sharedCache = new HashSet<>(); + } + + /** + * Initialize this load manager using the given PulsarService. Should be called only once, after invoking the + * default constructor. + * + * @param pulsar + * The service to initialize with. + */ + public void initialize(final PulsarService pulsar) { + adminCache = CacheBuilder.newBuilder().removalListener(new RemovalListener() { + public void onRemoval(RemovalNotification removal) { + removal.getValue().close(); + } + }).expireAfterAccess(1, TimeUnit.DAYS).build(new CacheLoader() { + @Override + public PulsarAdmin load(String key) throws Exception { + // key - broker name already is valid URL, has prefix "http://" + return new PulsarAdmin(new URL(key), pulsar.getConfiguration().getBrokerClientAuthenticationPlugin(), + pulsar.getConfiguration().getBrokerClientAuthenticationParameters()); + } + }); + + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), + LoadManager.LOADBALANCE_BROKERS_ROOT); + availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { + @Override + public void onUpdate(String path, Set data, Stat stat) { + if (log.isDebugEnabled()) { + log.debug("Update Received for path {}", path); + } + scheduler.submit(ModularLoadManagerImpl.this::updateAll); + } + }); + + brokerDataCache = new ZooKeeperDataCache(pulsar.getLocalZkCache()) { + @Override + public LocalBrokerData deserialize(String key, byte[] content) throws Exception { + return ObjectMapperFactory.getThreadLocal().readValue(content, LocalBrokerData.class); + } + }; + + brokerDataCache.registerListener(this); + + if (SystemUtils.IS_OS_LINUX) { + brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); + } else { + brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); + } + + conf = pulsar.getConfiguration(); + + // Initialize the default stats to assume for unseen bundles (hard-coded for now). + defaultStats.msgThroughputIn = DEFAULT_MESSAGE_THROUGHPUT; + defaultStats.msgThroughputOut = DEFAULT_MESSAGE_THROUGHPUT; + defaultStats.msgRateIn = DEFAULT_MESSAGE_RATE; + defaultStats.msgRateOut = DEFAULT_MESSAGE_RATE; + + localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + placementStrategy = ModularLoadManagerStrategy.create(conf); + policies = new SimpleResourceAllocationPolicies(pulsar); + this.pulsar = pulsar; + zkClient = pulsar.getZkClient(); + } + + /** + * Initialize this load manager. + * + * @param pulsar + * Client to construct this manager from. + */ + public ModularLoadManagerImpl(final PulsarService pulsar) { + this(); + initialize(pulsar); + } + + // Attempt to create a ZooKeeper path if it does not exist. + private static void createZPathIfNotExists(final ZooKeeper zkClient, final String path) throws Exception { + if (zkClient.exists(path, false) == null) { + try { + ZkUtils.createFullPathOptimistic(zkClient, path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + // Ignore if already exists. + } + } + } + + // Attempt to local the data for the given bundle in ZooKeeper. + // If it cannot be found, return the default bundle data. + private BundleData getBundleDataOrDefault(final String bundle) { + BundleData bundleData = null; + try { + final String bundleZPath = getBundleDataZooKeeperPath(bundle); + if (zkClient.exists(bundleZPath, null) != null) { + bundleData = readJson(zkClient.getData(bundleZPath, null, null), BundleData.class); + } + } catch (Exception e) { + log.warn("Error when trying to find bundle {} on zookeeper: {}", bundle, e); + } + if (bundleData == null) { + bundleData = new BundleData(NUM_SHORT_SAMPLES, NUM_LONG_SAMPLES, defaultStats); + } + return bundleData; + } + + // Get the ZooKeeper path for the given bundle full name. + private static String getBundleDataZooKeeperPath(final String bundle) { + return BUNDLE_DATA_ZPATH + "/" + bundle; + } + + // Use the Pulsar client to acquire the namespace bundle stats. + private Map getBundleStats() { + return pulsar.getBrokerService().getBundleStats(); + } + + // Use the thread local ObjectMapperFactory to read the given json data into an instance of the given class. + private static T readJson(final byte[] data, final Class clazz) throws IOException { + return ObjectMapperFactory.getThreadLocal().readValue(data, clazz); + } + + // Determine if the broker data requires an update by measuring the time + // past since the last update. + private boolean needBrokerDataUpdate() { + return System.currentTimeMillis() > localData.getLastUpdate() + + TimeUnit.MINUTES.toMillis(conf.getLoadBalancerReportUpdateMaxIntervalMinutes()); + } + + // Determine if the bundle data requires an update by measuring the time + // past since the last update. + private boolean needBundleDataUpdate() { + return System.currentTimeMillis() > lastBundleDataUpdate + + TimeUnit.MINUTES.toMillis(conf.getLoadBalancerResourceQuotaUpdateIntervalMinutes()); + } + + // Update both the broker data and the bundle data. + private void updateAll() { + updateAllBrokerData(); + updateBundleData(); + } + + // As the leader broker, update the broker data map in loadData by querying ZooKeeper for the broker data put there + // by each broker via updateLocalBrokerData. + private void updateAllBrokerData() { + try { + Set activeBrokers = availableActiveBrokers.get(); + final Map brokerDataMap = loadData.getBrokerData(); + for (String broker : activeBrokers) { + try { + String key = String.format("%s/%s", LoadManager.LOADBALANCE_BROKERS_ROOT, broker); + final LocalBrokerData localData = brokerDataCache.get(key) + .orElseThrow(KeeperException.NoNodeException::new); + + if (brokerDataMap.containsKey(broker)) { + // Replace previous local broker data. + brokerDataMap.get(broker).setLocalData(localData); + } else { + // Initialize BrokerData object for previously unseen + // brokers. + brokerDataMap.put(broker, new BrokerData(localData)); + } + } catch (Exception e) { + log.warn("Error reading broker data from cache for broker - [{}], [{}]", broker, e); + } + } + } catch (Exception e) { + log.warn("Error reading active brokers list from zookeeper while updating broker data [{}]", e); + } + } + + // As the leader broker, use the local broker data saved on ZooKeeper to update the bundle stats so that better load + // management decisions may be made. + private void updateBundleData() { + final Map bundleData = loadData.getBundleData(); + // Iterate over the broker data. + for (Map.Entry brokerEntry : loadData.getBrokerData().entrySet()) { + final String broker = brokerEntry.getKey(); + final BrokerData brokerData = brokerEntry.getValue(); + final Map statsMap = brokerData.getLocalData().getLastStats(); + + // Iterate over the last bundle stats available to the current + // broker to update the bundle data. + for (Map.Entry entry : statsMap.entrySet()) { + final String bundle = entry.getKey(); + final NamespaceBundleStats stats = entry.getValue(); + if (bundleData.containsKey(bundle)) { + // If we recognize the bundle, add these stats as a new + // sample. + bundleData.get(bundle).update(stats); + } else { + // Otherwise, attempt to find the bundle data on ZooKeeper. + // If it cannot be found, use the latest stats as the first + // sample. + BundleData currentBundleData = getBundleDataOrDefault(bundle); + currentBundleData.update(stats); + bundleData.put(bundle, currentBundleData); + } + } + + // Remove all loaded bundles from the preallocated maps. + final Map preallocatedBundleData = brokerData.getPreallocatedBundleData(); + if (preallocatedBundleData.containsKey(broker)) { + final Iterator> preallocatedIterator = preallocatedBundleData.entrySet() + .iterator(); + while (preallocatedIterator.hasNext()) { + final String bundle = preallocatedIterator.next().getKey(); + if (bundleData.containsKey(bundle)) { + preallocatedIterator.remove(); + preallocatedBundleToBroker.remove(bundle); + } + } + } + + // Using the newest data, update the aggregated time-average data + // for the current broker. + brokerData.getTimeAverageData().reset(statsMap.keySet(), bundleData, defaultStats); + } + } + + /** + * As any broker, disable the broker this manager is running on. + * + * @throws PulsarServerException + * If ZooKeeper failed to disable the broker. + */ + @Override + public void disableBroker() throws PulsarServerException { + if (StringUtils.isNotEmpty(brokerZnodePath)) { + try { + pulsar.getZkClient().delete(brokerZnodePath, -1); + } catch (Exception e) { + throw new PulsarServerException(e); + } + } + } + + /** + * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new + * brokers. + */ + @Override + public synchronized void doLoadShedding() { + for (LoadSheddingStrategy strategy : loadSheddingPipeline) { + final Map bundlesToUnload = strategy.findBundlesForUnloading(loadData, conf); + if (bundlesToUnload != null && !bundlesToUnload.isEmpty()) { + try { + for (Map.Entry entry : bundlesToUnload.entrySet()) { + final String bundle = entry.getKey(); + final String broker = entry.getValue(); + adminCache.get(broker).namespaces().unloadNamespaceBundle( + LoadManagerShared.getNamespaceNameFromBundleName(bundle), + LoadManagerShared.getBundleRangeFromBundleName(bundle)); + } + } catch (Exception e) { + log.warn("Error when trying to perform load shedding: {}", e); + } + return; + } + } + } + + /** + * As the leader broker, attempt to automatically detect and split hot namespace bundles. + */ + @Override + public void doNamespaceBundleSplit() { + // TODO? + } + + /** + * When the broker data ZooKeeper nodes are updated, update the broker data map. + */ + @Override + public void onUpdate(final String path, final LocalBrokerData data, final Stat stat) { + scheduler.submit(this::updateAll); + } + + /** + * As the leader broker, find a suitable broker for the assignment of the given bundle. + * + * @param serviceUnit + * ServiceUnitId for the bundle. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + @Override + public synchronized String selectBrokerForAssignment(final ServiceUnitId serviceUnit) { + // ?: Is it too inefficient to make this synchronized? If so, it may be + // a good idea to use weighted random + // or atomic data. + + final String bundle = serviceUnit.toString(); + if (preallocatedBundleToBroker.containsKey(bundle)) { + // If the given bundle is already in preallocated, return the + // selected broker. + return preallocatedBundleToBroker.get(bundle); + } + final BundleData data = loadData.getBundleData().computeIfAbsent(bundle, key -> getBundleDataOrDefault(bundle)); + brokerCandidateCache.clear(); + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, loadData.getBrokerData().keySet()); + + // Use the filter pipeline to finalize broker candidates. + for (BrokerFilter filter : filterPipeline) { + filter.filter(brokerCandidateCache, data, loadData, conf); + } + final String broker = placementStrategy.selectBroker(brokerCandidateCache, data, loadData, conf); + + // Add new bundle to preallocated. + loadData.getBrokerData().get(broker).getPreallocatedBundleData().put(bundle, data); + preallocatedBundleToBroker.put(bundle, broker); + return broker; + } + + /** + * As any broker, start the load manager. + * + * @throws PulsarServerException + * If an unexpected error prevented the load manager from being started. + */ + @Override + public void start() throws PulsarServerException { + try { + // Register the brokers in zk list + createZPathIfNotExists(zkClient, LoadManager.LOADBALANCE_BROKERS_ROOT); + + String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); + brokerZnodePath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; + final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + lookupServiceAddress; + updateLocalBrokerData(); + try { + ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerZnodePath, localData.getJsonBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); + } catch (Exception e) { + // Catching exception here to print the right error message + log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e); + throw e; + } + createZPathIfNotExists(zkClient, timeAverageZPath); + zkClient.setData(timeAverageZPath, (new TimeAverageBrokerData()).getJsonBytes(), -1); + updateAll(); + lastBundleDataUpdate = System.currentTimeMillis(); + } catch (Exception e) { + log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e); + throw new PulsarServerException(e); + } + } + + /** + * As any broker, stop the load manager. + * + * @throws PulsarServerException + * If an unexpected error occurred when attempting to stop the load manager. + */ + @Override + public void stop() throws PulsarServerException { + // Do nothing. + } + + /** + * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. + */ + @Override + public void updateLocalBrokerData() { + try { + final SystemResourceUsage systemResourceUsage = LoadManagerShared.getSystemResourceUsage(brokerHostUsage); + localData.update(systemResourceUsage, getBundleStats()); + } catch (Exception e) { + log.warn("Error when attempting to update local broker data: {}", e); + } + } + + /** + * As any broker, write the local broker data to ZooKeeper. + */ + @Override + public void writeBrokerDataOnZooKeeper() { + try { + if (needBrokerDataUpdate()) { + updateLocalBrokerData(); + zkClient.setData(brokerZnodePath, localData.getJsonBytes(), -1); + } + } catch (Exception e) { + log.warn("Error writing broker data on ZooKeeper: {}", e); + } + } + + /** + * As the leader broker, write bundle data aggregated from all brokers to ZooKeeper. + */ + @Override + public void writeBundleDataOnZooKeeper() { + if (needBundleDataUpdate()) { + updateBundleData(); + // Write the bundle data to ZooKeeper. + for (Map.Entry entry : loadData.getBundleData().entrySet()) { + final String bundle = entry.getKey(); + final BundleData data = entry.getValue(); + try { + final String zooKeeperPath = getBundleDataZooKeeperPath(bundle); + createZPathIfNotExists(zkClient, zooKeeperPath); + zkClient.setData(zooKeeperPath, data.getJsonBytes(), -1); + } catch (Exception e) { + log.warn("Error when writing data for bundle {} to ZooKeeper: {}", bundle, e); + } + } + // Write the time average broker data to ZooKeeper. + for (Map.Entry entry : loadData.getBrokerData().entrySet()) { + final String broker = entry.getKey(); + final TimeAverageBrokerData data = entry.getValue().getTimeAverageData(); + try { + final String zooKeeperPath = TIME_AVERAGE_BROKER_ZPATH + "/" + broker; + createZPathIfNotExists(zkClient, zooKeeperPath); + zkClient.setData(zooKeeperPath, data.getJsonBytes(), -1); + } catch (Exception e) { + log.warn("Error when writing time average broker data for {} to ZooKeeper: {}", broker, e); + } + } + } + } + +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java new file mode 100644 index 0000000000000..498083a8320a3 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java @@ -0,0 +1,106 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import java.util.Collections; +import java.util.List; + +import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.broker.PulsarService; +import com.yahoo.pulsar.broker.loadbalance.LoadManager; +import com.yahoo.pulsar.broker.loadbalance.ModularLoadManager; +import com.yahoo.pulsar.broker.loadbalance.ResourceUnit; +import com.yahoo.pulsar.broker.stats.Metrics; +import com.yahoo.pulsar.common.naming.ServiceUnitId; +import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; + +/** + * Wrapper class allowing classes of instance ModularLoadManager to be compatible with the interface LoadManager. + */ +public class ModularLoadManagerWrapper implements LoadManager { + private ModularLoadManager loadManager; + + public ModularLoadManagerWrapper(final ModularLoadManager loadManager) { + this.loadManager = loadManager; + } + + @Override + public void disableBroker() throws Exception { + loadManager.disableBroker(); + } + + @Override + public void doLoadShedding() { + loadManager.doLoadShedding(); + } + + @Override + public void doNamespaceBundleSplit() { + loadManager.doNamespaceBundleSplit(); + } + + @Override + public LoadReport generateLoadReport() { + loadManager.updateLocalBrokerData(); + return null; + } + + @Override + public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { + return new SimpleResourceUnit(String.format("http://%s", loadManager.selectBrokerForAssignment(serviceUnit)), + new PulsarResourceDescription()); + } + + @Override + public List getLoadBalancingMetrics() { + return Collections.emptyList(); + } + + @Override + public void initialize(final PulsarService pulsar) { + loadManager.initialize(pulsar); + } + + @Override + public boolean isCentralized() { + return true; + } + + @Override + public void setLoadReportForceUpdateFlag() { + + } + + @Override + public void start() throws PulsarServerException { + loadManager.start(); + } + + @Override + public void stop() throws PulsarServerException { + loadManager.stop(); + } + + @Override + public void writeLoadReportOnZookeeper() { + loadManager.writeBrokerDataOnZooKeeper(); + } + + @Override + public void writeResourceQuotasToZooKeeper() { + loadManager.writeBundleDataOnZooKeeper(); + } +} diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java index fff4201d3636d..79c6bdfbc9d8c 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java @@ -82,7 +82,7 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListener { private static final Logger log = LoggerFactory.getLogger(SimpleLoadManagerImpl.class); - private final SimpleResourceAllocationPolicies policies; + private SimpleResourceAllocationPolicies policies; private PulsarService pulsar; // average JVM heap usage for @@ -98,6 +98,16 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene // load balancing metrics private AtomicReference> loadBalancingMetrics = new AtomicReference<>(); + // Cache of brokers to be used in applying policies and determining final candidates. + private final Set brokerCandidateCache; + + // Other policy selection caches. + private final Set availableBrokersCache; + + // Caches for bundle gains and losses. + private final Set bundleGainsCache; + private final Set bundleLossesCache; + // CPU usage per msg/sec private double realtimeCpuLoadFactor = 0.025; // memory usage per 500 (topics + producers + consumers) @@ -120,21 +130,20 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private static final long RESOURCE_QUOTA_MIN_BANDWIDTH_IN = 10000; private static final long RESOURCE_QUOTA_MIN_BANDWIDTH_OUT = 10000; private static final long RESOURCE_QUOTA_MIN_MEMORY = 2; - private static final long RESOURCE_QUOTA_MAX_MSGRATE_IN = 5000; - private static final long RESOURCE_QUOTA_MAX_MSGRATE_OUT = 5000; + private static final long RESOURCE_QUOTA_MAX_MSGRATE_IN = 0; + private static final long RESOURCE_QUOTA_MAX_MSGRATE_OUT = 0; private static final long RESOURCE_QUOTA_MAX_BANDWIDTH_IN = 1000000; private static final long RESOURCE_QUOTA_MAX_BANDWIDTH_OUT = 1000000; private static final long RESOURCE_QUOTA_MAX_MEMORY = 200; private final PlacementStrategy placementStrategy; - private final ZooKeeperDataCache loadReportCacheZk; - private final ZooKeeperDataCache> dynamicConfigurationCache; - private final BrokerHostUsage brokerHostUsage; - private final LoadingCache adminCache; - private final LoadingCache unloadedHotNamespaceCache; + private ZooKeeperDataCache loadReportCacheZk; + private ZooKeeperDataCache> dynamicConfigurationCache; + private BrokerHostUsage brokerHostUsage; + private LoadingCache adminCache; + private LoadingCache unloadedHotNamespaceCache; - public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/brokers"; public static final String LOADBALANCER_DYNAMIC_SETTING_STRATEGY_ZPATH = "/loadbalance/settings/strategy"; private static final String LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_CPU_ZPATH = "/loadbalance/settings/load_factor_cpu"; private static final String LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_MEM_ZPATH = "/loadbalance/settings/load_factor_mem"; @@ -152,10 +161,11 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene public static final String LOADBALANCER_STRATEGY_LLS = "leastLoadedServer"; public static final String LOADBALANCER_STRATEGY_RAND = "weightedRandomSelection"; + public static final String LOADBALANCER_STRATEGY_LEAST_MSG = "leastMsgPerSecond"; private String brokerZnodePath; private final ScheduledExecutorService scheduler; - private final ZooKeeperChildrenCache availableActiveBrokers; + private ZooKeeperChildrenCache availableActiveBrokers; private static final long MBytes = 1024 * 1024; // update LoadReport at most every 5 seconds @@ -167,8 +177,9 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene // flag to force update load report private boolean forceLoadReportUpdate = false; - public SimpleLoadManagerImpl(PulsarService pulsar) { - this.policies = new SimpleResourceAllocationPolicies(pulsar); + // Perform initializations which may be done without a PulsarService. + public SimpleLoadManagerImpl() { + scheduler = Executors.newScheduledThreadPool(1); this.sortedRankings.set(new TreeMap<>()); this.currentLoadReports = new HashMap<>(); this.resourceUnitRankings = new HashMap<>(); @@ -176,13 +187,22 @@ public SimpleLoadManagerImpl(PulsarService pulsar) { this.realtimeResourceQuotas.set(new HashMap<>()); this.realtimeAvgResourceQuota = new ResourceQuota(); placementStrategy = new WRRPlacementStrategy(); - lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + bundleGainsCache = new HashSet<>(); + bundleLossesCache = new HashSet<>(); + brokerCandidateCache = new HashSet<>(); + availableBrokersCache = new HashSet<>(); + } + + @Override + public void initialize(final PulsarService pulsar) { if (SystemUtils.IS_OS_LINUX) { brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); } else { brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); } + this.policies = new SimpleResourceAllocationPolicies(pulsar); + lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); loadReportCacheZk = new ZooKeeperDataCache(pulsar.getLocalZkCache()) { @Override public LoadReport deserialize(String key, byte[] content) throws Exception { @@ -205,17 +225,18 @@ public void onRemoval(RemovalNotification removal) { public PulsarAdmin load(String key) throws Exception { // key - broker name already is valid URL, has prefix "http://" return new PulsarAdmin(new URL(key), pulsar.getConfiguration().getBrokerClientAuthenticationPlugin(), - pulsar.getConfiguration().getBrokerClientAuthenticationParameters()); + pulsar.getConfiguration().getBrokerClientAuthenticationParameters()); } }); int entryExpiryTime = (int) pulsar.getConfiguration().getLoadBalancerSheddingGracePeriodMinutes(); unloadedHotNamespaceCache = CacheBuilder.newBuilder().expireAfterWrite(entryExpiryTime, TimeUnit.MINUTES) - .build(new CacheLoader() { - @Override - public Long load(String key) throws Exception { - return System.currentTimeMillis(); - } - }); + .build(new CacheLoader() { + @Override + public Long load(String key) throws Exception { + return System.currentTimeMillis(); + } + }); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LOADBALANCE_BROKERS_ROOT); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override @@ -226,10 +247,14 @@ public void onUpdate(String path, Set data, Stat stat) { scheduler.submit(SimpleLoadManagerImpl.this::updateRanking); } }); - scheduler = Executors.newScheduledThreadPool(1); this.pulsar = pulsar; } + public SimpleLoadManagerImpl(PulsarService pulsar) { + this(); + initialize(pulsar); + } + @Override public void start() throws PulsarServerException { try { @@ -238,12 +263,11 @@ public void start() throws PulsarServerException { if (pulsar.getZkClient().exists(LOADBALANCE_BROKERS_ROOT, false) == null) { try { ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), LOADBALANCE_BROKERS_ROOT, new byte[0], - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } catch (KeeperException.NodeExistsException e) { // ignore the exception, node might be present already } } - String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); brokerZnodePath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; LoadReport loadReport = null; @@ -259,7 +283,7 @@ public void start() throws PulsarServerException { } try { ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerZnodePath, - loadReportJson.getBytes(Charsets.UTF_8), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); + loadReportJson.getBytes(Charsets.UTF_8), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); } catch (Exception e) { // Catching excption here to print the right error message log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e); @@ -273,11 +297,11 @@ public void start() throws PulsarServerException { this.realtimeAvgResourceQuota = pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota(); this.lastResourceQuotaUpdateTimestamp = System.currentTimeMillis(); this.realtimeCpuLoadFactor = getDynamicConfigurationDouble( - LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_CPU_ZPATH, SETTING_NAME_LOAD_FACTOR_CPU, - this.realtimeCpuLoadFactor); + LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_CPU_ZPATH, SETTING_NAME_LOAD_FACTOR_CPU, + this.realtimeCpuLoadFactor); this.realtimeMemoryLoadFactor = getDynamicConfigurationDouble( - LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_MEM_ZPATH, SETTING_NAME_LOAD_FACTOR_MEM, - this.realtimeMemoryLoadFactor); + LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_MEM_ZPATH, SETTING_NAME_LOAD_FACTOR_MEM, + this.realtimeMemoryLoadFactor); } catch (Exception e) { log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e); throw new PulsarServerException(e); @@ -306,7 +330,7 @@ private void setDynamicConfigurationToZK(String zkPath, Map sett pulsar.getZkClient().setData(zkPath, settingBytes, -1); } else { ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), zkPath, settingBytes, Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); + CreateMode.PERSISTENT); } } catch (Exception e) { log.warn("Got exception when writing to ZooKeeper path [{}]:", zkPath, e); @@ -351,8 +375,9 @@ private boolean getDynamicConfigurationBoolean(String zkPath, String settingName private String getLoadBalancerPlacementStrategy() { String strategy = this.getDynamicConfigurationFromZK(LOADBALANCER_DYNAMIC_SETTING_STRATEGY_ZPATH, - SETTING_NAME_STRATEGY, pulsar.getConfiguration().getLoadBalancerPlacementStrategy()); - if (!LOADBALANCER_STRATEGY_LLS.equals(strategy) && !LOADBALANCER_STRATEGY_RAND.equals(strategy)) { + SETTING_NAME_STRATEGY, pulsar.getConfiguration().getLoadBalancerPlacementStrategy()); + if (!LOADBALANCER_STRATEGY_LLS.equals(strategy) && !LOADBALANCER_STRATEGY_RAND.equals(strategy) + && !LOADBALANCER_STRATEGY_LEAST_MSG.equals(strategy)) { strategy = LOADBALANCER_STRATEGY_RAND; } return strategy; @@ -360,42 +385,45 @@ private String getLoadBalancerPlacementStrategy() { private double getCpuLoadFactorFromZK(double defaultValue) { return getDynamicConfigurationDouble(LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_CPU_ZPATH, - SETTING_NAME_LOAD_FACTOR_CPU, defaultValue); + SETTING_NAME_LOAD_FACTOR_CPU, defaultValue); } private double getMemoryLoadFactorFromZK(double defaultValue) { return getDynamicConfigurationDouble(LOADBALANCER_DYNAMIC_SETTING_LOAD_FACTOR_MEM_ZPATH, - SETTING_NAME_LOAD_FACTOR_MEM, defaultValue); + SETTING_NAME_LOAD_FACTOR_MEM, defaultValue); } @Override public boolean isCentralized() { String strategy = this.getLoadBalancerPlacementStrategy(); - return (strategy.equals(LOADBALANCER_STRATEGY_LLS)); + if (strategy.equals(LOADBALANCER_STRATEGY_LLS) || strategy.equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { + return true; + } + return false; } private long getLoadBalancerBrokerUnderloadedThresholdPercentage() { return (long) this.getDynamicConfigurationDouble(LOADBALANCER_DYNAMIC_SETTING_UNDERLOAD_THRESHOLD_ZPATH, - SETTING_NAME_UNDERLOAD_THRESHOLD, - pulsar.getConfiguration().getLoadBalancerBrokerUnderloadedThresholdPercentage()); + SETTING_NAME_UNDERLOAD_THRESHOLD, + pulsar.getConfiguration().getLoadBalancerBrokerUnderloadedThresholdPercentage()); } private long getLoadBalancerBrokerOverloadedThresholdPercentage() { return (long) this.getDynamicConfigurationDouble(LOADBALANCER_DYNAMIC_SETTING_OVERLOAD_THRESHOLD_ZPATH, - SETTING_NAME_OVERLOAD_THRESHOLD, - pulsar.getConfiguration().getLoadBalancerBrokerOverloadedThresholdPercentage()); + SETTING_NAME_OVERLOAD_THRESHOLD, + pulsar.getConfiguration().getLoadBalancerBrokerOverloadedThresholdPercentage()); } private long getLoadBalancerBrokerComfortLoadThresholdPercentage() { return (long) this.getDynamicConfigurationDouble(LOADBALANCER_DYNAMIC_SETTING_COMFORT_LOAD_THRESHOLD_ZPATH, - SETTING_NAME_COMFORTLOAD_THRESHOLD, - pulsar.getConfiguration().getLoadBalancerBrokerComfortLoadLevelPercentage()); + SETTING_NAME_COMFORTLOAD_THRESHOLD, + pulsar.getConfiguration().getLoadBalancerBrokerComfortLoadLevelPercentage()); } private boolean getLoadBalancerAutoBundleSplitEnabled() { return this.getDynamicConfigurationBoolean(LOADBALANCER_DYNAMIC_SETTING_AUTO_BUNDLE_SPLIT_ENABLED, - SETTING_NAME_AUTO_BUNDLE_SPLIT_ENABLED, - pulsar.getConfiguration().getLoadBalancerAutoBundleSplitEnabled()); + SETTING_NAME_AUTO_BUNDLE_SPLIT_ENABLED, + pulsar.getConfiguration().getLoadBalancerAutoBundleSplitEnabled()); } /* @@ -456,19 +484,19 @@ private double timeSmoothValue(double oldValue, double newSample, double minValu } private ResourceQuota timeSmoothQuota(ResourceQuota oldQuota, double msgRateIn, double msgRateOut, - double bandwidthIn, double bandwidthOut, double memory, long timePast) { + double bandwidthIn, double bandwidthOut, double memory, long timePast) { if (oldQuota.getDynamic()) { ResourceQuota newQuota = new ResourceQuota(); newQuota.setMsgRateIn(timeSmoothValue(oldQuota.getMsgRateIn(), msgRateIn, RESOURCE_QUOTA_MIN_MSGRATE_IN, - RESOURCE_QUOTA_MAX_MSGRATE_IN, timePast)); + RESOURCE_QUOTA_MAX_MSGRATE_IN, timePast)); newQuota.setMsgRateOut(timeSmoothValue(oldQuota.getMsgRateOut(), msgRateOut, RESOURCE_QUOTA_MIN_MSGRATE_OUT, - RESOURCE_QUOTA_MAX_MSGRATE_OUT, timePast)); + RESOURCE_QUOTA_MAX_MSGRATE_OUT, timePast)); newQuota.setBandwidthIn(timeSmoothValue(oldQuota.getBandwidthIn(), bandwidthIn, - RESOURCE_QUOTA_MIN_BANDWIDTH_IN, RESOURCE_QUOTA_MAX_BANDWIDTH_IN, timePast)); + RESOURCE_QUOTA_MIN_BANDWIDTH_IN, RESOURCE_QUOTA_MAX_BANDWIDTH_IN, timePast)); newQuota.setBandwidthOut(timeSmoothValue(oldQuota.getBandwidthOut(), bandwidthOut, - RESOURCE_QUOTA_MIN_BANDWIDTH_OUT, RESOURCE_QUOTA_MAX_BANDWIDTH_OUT, timePast)); + RESOURCE_QUOTA_MIN_BANDWIDTH_OUT, RESOURCE_QUOTA_MAX_BANDWIDTH_OUT, timePast)); newQuota.setMemory(timeSmoothValue(oldQuota.getMemory(), memory, RESOURCE_QUOTA_MIN_MEMORY, - RESOURCE_QUOTA_MAX_MEMORY, timePast)); + RESOURCE_QUOTA_MAX_MEMORY, timePast)); return newQuota; } else { return oldQuota; @@ -505,7 +533,7 @@ private synchronized void updateRealtimeResourceQuota() { totalBundles++; NamespaceBundleStats stats = statsEntry.getValue(); totalMemGroups += (1 - + (stats.topics + stats.producerCount + stats.consumerCount) / memObjectGroupSize); + + (stats.topics + stats.producerCount + stats.consumerCount) / memObjectGroupSize); totalBandwidthIn += stats.msgThroughputIn; totalBandwidthOut += stats.msgThroughputOut; } @@ -522,18 +550,18 @@ private synchronized void updateRealtimeResourceQuota() { this.lastResourceQuotaUpdateTimestamp = loadReportTimestamp; if (totalMsgRate > 1000 && totalMemGroups > 30) { this.realtimeCpuLoadFactor = timeSmoothValue(this.realtimeCpuLoadFactor, totalCpuUsage / totalMsgRate, - RESOURCE_QUOTA_MIN_CPU_FACTOR, RESOURCE_QUOTA_MAX_CPU_FACTOR, timePast); + RESOURCE_QUOTA_MIN_CPU_FACTOR, RESOURCE_QUOTA_MAX_CPU_FACTOR, timePast); this.realtimeMemoryLoadFactor = timeSmoothValue(this.realtimeMemoryLoadFactor, - totalMemoryUsage / totalMemGroups, RESOURCE_QUOTA_MIN_MEM_FACTOR, RESOURCE_QUOTA_MAX_MEM_FACTOR, - timePast); + totalMemoryUsage / totalMemGroups, RESOURCE_QUOTA_MIN_MEM_FACTOR, RESOURCE_QUOTA_MAX_MEM_FACTOR, + timePast); } // calculate average bundle if (totalBundles > 30 && this.realtimeAvgResourceQuota.getDynamic()) { ResourceQuota oldQuota = this.realtimeAvgResourceQuota; ResourceQuota newQuota = timeSmoothQuota(oldQuota, totalMsgRateIn / totalBundles, - totalMsgRateOut / totalBundles, totalBandwidthIn / totalBundles, - totalBandwidthOut / totalBundles, totalMemoryUsage / totalBundles, timePast); + totalMsgRateOut / totalBundles, totalBandwidthIn / totalBundles, + totalBandwidthOut / totalBundles, totalMemoryUsage / totalBundles, timePast); this.realtimeAvgResourceQuota = newQuota; } @@ -551,12 +579,12 @@ private synchronized void updateRealtimeResourceQuota() { String bundle = statsEntry.getKey(); NamespaceBundleStats stats = statsEntry.getValue(); long memGroupCount = (1 - + (stats.topics + stats.producerCount + stats.consumerCount) / memObjectGroupSize); + + (stats.topics + stats.producerCount + stats.consumerCount) / memObjectGroupSize); double newMemoryQuota = memGroupCount * this.realtimeMemoryLoadFactor; ResourceQuota oldQuota = getResourceQuota(bundle); ResourceQuota newQuota = timeSmoothQuota(oldQuota, stats.msgRateIn, stats.msgRateOut, - stats.msgThroughputIn, stats.msgThroughputOut, newMemoryQuota, timePast); + stats.msgThroughputIn, stats.msgThroughputOut, newMemoryQuota, timePast); newQuotas.put(bundle, newQuota); } } @@ -567,21 +595,19 @@ private synchronized void updateRealtimeResourceQuota() { private void compareAndWriteQuota(String bundle, ResourceQuota oldQuota, ResourceQuota newQuota) throws Exception { boolean needUpdate = true; if (!oldQuota.getDynamic() || (Math - .abs(newQuota.getMsgRateIn() - oldQuota.getMsgRateIn()) < RESOURCE_QUOTA_MIN_MSGRATE_IN - && Math.abs(newQuota.getMsgRateOut() - oldQuota.getMsgRateOut()) < RESOURCE_QUOTA_MIN_MSGRATE_OUT - && Math.abs(newQuota.getBandwidthIn() - oldQuota.getBandwidthOut()) < RESOURCE_QUOTA_MIN_BANDWIDTH_IN - && Math.abs(newQuota.getBandwidthOut() - oldQuota.getBandwidthOut()) < RESOURCE_QUOTA_MIN_BANDWIDTH_OUT - && Math.abs(newQuota.getMemory() - oldQuota.getMemory()) < RESOURCE_QUOTA_MIN_MEMORY)) { + .abs(newQuota.getMsgRateIn() - oldQuota.getMsgRateIn()) < RESOURCE_QUOTA_MIN_MSGRATE_IN + && Math.abs(newQuota.getMsgRateOut() - oldQuota.getMsgRateOut()) < RESOURCE_QUOTA_MIN_MSGRATE_OUT + && Math.abs(newQuota.getBandwidthIn() - oldQuota.getBandwidthOut()) < RESOURCE_QUOTA_MIN_BANDWIDTH_IN + && Math.abs(newQuota.getBandwidthOut() - oldQuota.getBandwidthOut()) < RESOURCE_QUOTA_MIN_BANDWIDTH_OUT + && Math.abs(newQuota.getMemory() - oldQuota.getMemory()) < RESOURCE_QUOTA_MIN_MEMORY)) { needUpdate = false; } if (needUpdate) { - if (log.isDebugEnabled()) { - log.debug(String.format( + log.info(String.format( "Update quota %s - msgRateIn: %.1f, msgRateOut: %.1f, bandwidthIn: %.1f, bandwidthOut: %.1f, memory: %.1f", (bundle == null) ? "default" : bundle, newQuota.getMsgRateIn(), newQuota.getMsgRateOut(), newQuota.getBandwidthIn(), newQuota.getBandwidthOut(), newQuota.getMemory())); - } if (bundle == null) { pulsar.getLocalZkCacheService().getResourceQuotaCache().setDefaultQuota(newQuota); @@ -641,60 +667,62 @@ private synchronized void doLoadRanking() { String strategy = this.getLoadBalancerPlacementStrategy(); log.info("doLoadRanking - load balancing strategy: {}", strategy); if (!currentLoadReports.isEmpty()) { - synchronized (resourceUnitRankings) { - Map> newSortedRankings = Maps.newTreeMap(); - Map newResourceUnitRankings = new HashMap<>(); - - for (Map.Entry entry : currentLoadReports.entrySet()) { - ResourceUnit resourceUnit = entry.getKey(); - LoadReport loadReport = entry.getValue(); - - // calculate rankings - Set loadedBundles = loadReport.getBundles(); - Set preAllocatedBundles = null; - if (resourceUnitRankings.containsKey(resourceUnit)) { - preAllocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles(); - preAllocatedBundles.removeAll(loadedBundles); - } else { - preAllocatedBundles = new HashSet<>(); - } - ResourceQuota allocatedQuota = getTotalAllocatedQuota(loadedBundles); - ResourceQuota preAllocatedQuota = getTotalAllocatedQuota(preAllocatedBundles); - ResourceUnitRanking ranking = new ResourceUnitRanking(loadReport.getSystemResourceUsage(), + + Map> newSortedRankings = Maps.newTreeMap(); + Map newResourceUnitRankings = new HashMap<>(); + + for (Map.Entry entry : currentLoadReports.entrySet()) { + ResourceUnit resourceUnit = entry.getKey(); + LoadReport loadReport = entry.getValue(); + + // calculate rankings + Set loadedBundles = loadReport.getBundles(); + Set preAllocatedBundles = null; + if (resourceUnitRankings.containsKey(resourceUnit)) { + preAllocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles(); + preAllocatedBundles.removeAll(loadedBundles); + } else { + preAllocatedBundles = new HashSet<>(); + } + ResourceQuota allocatedQuota = getTotalAllocatedQuota(loadedBundles); + ResourceQuota preAllocatedQuota = getTotalAllocatedQuota(preAllocatedBundles); + ResourceUnitRanking ranking = new ResourceUnitRanking(loadReport.getSystemResourceUsage(), loadedBundles, allocatedQuota, preAllocatedBundles, preAllocatedQuota); - newResourceUnitRankings.put(resourceUnit, ranking); + newResourceUnitRankings.put(resourceUnit, ranking); - // generated sorted ranking - double loadPercentage = ranking.getEstimatedLoadPercentage(); - long maxCapacity = ranking.estimateMaxCapacity( - pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); - long finalRank = 0; - if (strategy.equals(LOADBALANCER_STRATEGY_LLS)) { - finalRank = (long) loadPercentage; - } else { - double idleRatio = (100 - loadPercentage) / 100; - finalRank = (long) (maxCapacity * idleRatio * idleRatio); - } + // generated sorted ranking + double loadPercentage = ranking.getEstimatedLoadPercentage(); + long maxCapacity = ranking + .estimateMaxCapacity(pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); + long finalRank = 0; + if (strategy.equals(LOADBALANCER_STRATEGY_LLS)) { + finalRank = (long) loadPercentage; + } else if (strategy.equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { + finalRank = (long) ranking.getEstimatedMessageRate(); + } else { + double idleRatio = (100 - loadPercentage) / 100; + finalRank = (long) (maxCapacity * idleRatio * idleRatio); + } - if (!newSortedRankings.containsKey(finalRank)) { - newSortedRankings.put(finalRank, new HashSet()); - } - newSortedRankings.get(finalRank).add(entry.getKey()); - if (log.isDebugEnabled()) { - log.debug("Added Resource Unit [{}] with Rank [{}]", entry.getKey().getResourceId(), finalRank); - } + if (!newSortedRankings.containsKey(finalRank)) { + newSortedRankings.put(finalRank, new HashSet()); + } + newSortedRankings.get(finalRank).add(entry.getKey()); + if (log.isDebugEnabled()) { + log.debug("Added Resource Unit [{}] with Rank [{}]", entry.getKey().getResourceId(), finalRank); + } - // update metrics - if (resourceUnit.getResourceId().contains(hostname)) { - updateLoadBalancingMetrics(hostname, finalRank, ranking); - } + // update metrics + if (resourceUnit.getResourceId().contains(hostname)) { + updateLoadBalancingMetrics(hostname, finalRank, ranking); } - this.sortedRankings.set(newSortedRankings); - this.resourceUnitRankings = newResourceUnitRankings; } + this.sortedRankings.set(newSortedRankings); + this.resourceUnitRankings = newResourceUnitRankings; + } else { log.info("Leader broker[{}] No ResourceUnits to rank this run, Using Old Ranking", - pulsar.getWebServiceAddress()); + pulsar.getWebServiceAddress()); } } @@ -739,7 +767,8 @@ private void updateLoadBalancingMetrics(String hostname, long finalRank, Resourc * preAllocatedQuota into calculation; 3) Everything (preAllocatedBundles and preAllocatedQuotas) will get reset in * load ranking. */ - private ResourceUnit findBrokerForPlacement(Multimap candidates, ServiceUnitId serviceUnit) { + private synchronized ResourceUnit findBrokerForPlacement(Multimap candidates, + ServiceUnitId serviceUnit) { long underloadThreshold = this.getLoadBalancerBrokerUnderloadedThresholdPercentage(); long overloadThreshold = this.getLoadBalancerBrokerOverloadedThresholdPercentage(); ResourceQuota defaultQuota = pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota(); @@ -749,182 +778,133 @@ private ResourceUnit findBrokerForPlacement(Multimap candida ResourceUnit idleRU = null; ResourceUnit maxAvailableRU = null; ResourceUnit randomRU = null; - ResourceUnit selectedRU = null; + ResourceUnitRanking selectedRanking = null; String serviceUnitId = serviceUnit.toString(); - synchronized (resourceUnitRankings) { - long randomBrokerIndex = (candidates.size() > 0) ? (this.brokerRotationCursor % candidates.size()) : 0; - // find the least loaded & not-idle broker - for (Map.Entry candidateOwner : candidates.entries()) { - ResourceUnit candidate = candidateOwner.getValue(); - randomBrokerIndex--; - - // skip broker which is not ranked. this should never happen except in unit test - if (!resourceUnitRankings.containsKey(candidate)) { - continue; - } + // If the ranking is expected to be in the range [0,100] (which is the case for LOADBALANCER_STRATEGY_LLS), + // the ranks are bounded. Otherwise (as is the case in LOADBALANCER_STRATEGY_LEAST_MSG, the ranks are simply + // the total message rate which is in the range [0,Infinity) so they are unbounded. The + // "boundedness" affects how two ranks are compared to see which one is better + boolean unboundedRanks = getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LEAST_MSG); + long randomBrokerIndex = (candidates.size() > 0) ? (this.brokerRotationCursor % candidates.size()) : 0; + + // find the least loaded & not-idle broker + for (Map.Entry candidateOwner : candidates.entries()) { + ResourceUnit candidate = candidateOwner.getValue(); + randomBrokerIndex--; + + // skip broker which is not ranked. this should never happen except in unit test + if (!resourceUnitRankings.containsKey(candidate)) { + continue; + } - // check if this ServiceUnit is already pre-allocated - String resourceUnitId = candidate.getResourceId(); - ResourceUnitRanking ranking = resourceUnitRankings.get(candidate); - if (ranking.isServiceUnitPreAllocated(serviceUnitId)) { - return candidate; - } + // check if this ServiceUnit is already pre-allocated + String resourceUnitId = candidate.getResourceId(); + ResourceUnitRanking ranking = resourceUnitRankings.get(candidate); + if (ranking.isServiceUnitPreAllocated(serviceUnitId)) { + return candidate; + } - // check if this ServiceUnit is already loaded - if (ranking.isServiceUnitLoaded(serviceUnitId)) { - ranking.removeLoadedServiceUnit(serviceUnitId, this.getResourceQuota(serviceUnitId)); - } + // check if this ServiceUnit is already loaded + if (ranking.isServiceUnitLoaded(serviceUnitId)) { + ranking.removeLoadedServiceUnit(serviceUnitId, this.getResourceQuota(serviceUnitId)); + } - // record a random broker - if (randomBrokerIndex < 0 && randomRU == null) { - randomRU = candidate; - } + // record a random broker + if (randomBrokerIndex < 0 && randomRU == null) { + randomRU = candidate; + } - // check the available capacity - double loadPercentage = ranking.getEstimatedLoadPercentage(); - double availablePercentage = Math.max(0, (100 - loadPercentage) / 100); - long availability = (long) (ranking.estimateMaxCapacity(defaultQuota) * availablePercentage); - if (availability > maxAvailability) { - maxAvailability = availability; - maxAvailableRU = candidate; - } + // check the available capacity + double loadPercentage = ranking.getEstimatedLoadPercentage(); + double availablePercentage = Math.max(0, (100 - loadPercentage) / 100); + long availability = (long) (ranking.estimateMaxCapacity(defaultQuota) * availablePercentage); + if (availability > maxAvailability) { + maxAvailability = availability; + maxAvailableRU = candidate; + } - // check the load percentage - if (ranking.isIdle()) { - if (idleRU == null) { - idleRU = candidate; - } + // check the load percentage + if (ranking.isIdle()) { + if (idleRU == null) { + idleRU = candidate; + } + } else { + if (selectedRU == null) { + selectedRU = candidate; + selectedRanking = ranking; + minLoadPercentage = loadPercentage; } else { - if (selectedRU == null) { + if ((unboundedRanks ? ranking.compareMessageRateTo(selectedRanking) + : ranking.compareTo(selectedRanking)) < 0) { + minLoadPercentage = loadPercentage; selectedRU = candidate; selectedRanking = ranking; - minLoadPercentage = loadPercentage; - } else { - if (ranking.compareTo(selectedRanking) < 0) { - minLoadPercentage = loadPercentage; - selectedRU = candidate; - selectedRanking = ranking; - } } } } + } - if ((minLoadPercentage > underloadThreshold && idleRU != null) || selectedRU == null) { - // assigned to idle broker is the least loaded broker already have optimum load (which means NOT - // underloaded), or all brokers are idle - selectedRU = idleRU; - } else if (minLoadPercentage >= 100.0 && randomRU != null) { - // all brokers are full, assign to a random one - selectedRU = randomRU; - } else if (minLoadPercentage > overloadThreshold) { - // assign to the broker with maximum available capacity if all brokers are overloaded - selectedRU = maxAvailableRU; - } + if ((minLoadPercentage > underloadThreshold && idleRU != null) || selectedRU == null) { + // assigned to idle broker is the least loaded broker already have optimum load (which means NOT + // underloaded), or all brokers are idle + selectedRU = idleRU; + } else if (minLoadPercentage >= 100.0 && randomRU != null && !unboundedRanks) { + // all brokers are full, assign to a random one + selectedRU = randomRU; + } else if (minLoadPercentage > overloadThreshold && !unboundedRanks) { + // assign to the broker with maximum available capacity if all brokers are overloaded + selectedRU = maxAvailableRU; + } - // re-calculate load level for selected broker - if (selectedRU != null) { - this.brokerRotationCursor = (this.brokerRotationCursor + 1) % 1000000; - ResourceUnitRanking ranking = resourceUnitRankings.get(selectedRU); - String loadPercentageDesc = ranking.getEstimatedLoadPercentageString(); - log.info("Assign {} to {} with ({}).", serviceUnitId, selectedRU.getResourceId(), loadPercentageDesc); - if (!ranking.isServiceUnitPreAllocated(serviceUnitId)) { - ResourceQuota quota = this.getResourceQuota(serviceUnitId); - ranking.addPreAllocatedServiceUnit(serviceUnitId, quota); - } + // re-calculate load level for selected broker + if (selectedRU != null) { + this.brokerRotationCursor = (this.brokerRotationCursor + 1) % 1000000; + ResourceUnitRanking ranking = resourceUnitRankings.get(selectedRU); + String loadPercentageDesc = ranking.getEstimatedLoadPercentageString(); + log.info("Assign {} to {} with ({}).", serviceUnitId, selectedRU.getResourceId(), loadPercentageDesc); + if (!ranking.isServiceUnitPreAllocated(serviceUnitId)) { + ResourceQuota quota = this.getResourceQuota(serviceUnitId); + ranking.addPreAllocatedServiceUnit(serviceUnitId, quota); + resourceUnitRankings.put(selectedRU, ranking); } } return selectedRU; } - private Multimap getFinalCandidatesWithPolicy(NamespaceName namespace, - Multimap primaries, Multimap shared) { - Multimap finalCandidates = TreeMultimap.create(); - // if not enough primary then it should be union of primaries and secondaries - finalCandidates.putAll(primaries); - if (policies.shouldFailoverToSecondaries(namespace, primaries.size())) { - log.debug( - "Not enough of primaries [{}] available for namespace - [{}], " - + "adding shared [{}] as possible candidate owners", - primaries.size(), namespace.toString(), shared.size()); - finalCandidates.putAll(shared); - } - return finalCandidates; - } - - private Multimap getFinalCandidatesNoPolicy(Multimap shared) { - Multimap finalCandidates = TreeMultimap.create(); - - finalCandidates.putAll(shared); - return finalCandidates; - } - private Multimap getFinalCandidates(ServiceUnitId serviceUnit, - Map> availableBrokers) { - // need multimap or at least set of RUs - Multimap matchedPrimaries = TreeMultimap.create(); - Multimap matchedShared = TreeMultimap.create(); - - NamespaceName namespace = serviceUnit.getNamespaceObject(); - boolean isIsolationPoliciesPresent = policies.IsIsolationPoliciesPresent(namespace); - if (isIsolationPoliciesPresent) { - log.debug("Isolation Policies Present for namespace - [{}]", namespace.toString()); - } - for (Map.Entry> entry : availableBrokers.entrySet()) { - for (ResourceUnit ru : entry.getValue()) { - log.debug("Considering Resource Unit [{}] with Rank [{}] for serviceUnit [{}]", ru.getResourceId(), - entry.getKey(), serviceUnit); - URL brokerUrl = null; - try { - brokerUrl = new URL(String.format(ru.getResourceId())); - } catch (MalformedURLException e) { - log.error("Unable to parse brokerUrl from ResourceUnitId - [{}]", e); - continue; + Map> availableBrokers) { + synchronized (brokerCandidateCache) { + final Multimap result = TreeMultimap.create(); + availableBrokersCache.clear(); + for (final Set resourceUnits : availableBrokers.values()) { + for (final ResourceUnit resourceUnit : resourceUnits) { + availableBrokersCache.add(resourceUnit.getResourceId().replace("http://", "")); } - // 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())) { - matchedPrimaries.put(entry.getKey(), ru); - if (log.isDebugEnabled()) { - log.debug( - "Added Primary Broker - [{}] as possible Candidates for" - + " namespace - [{}] with policies", - brokerUrl.getHost(), namespace.toString()); - } - } else if (policies.isSharedBroker(brokerUrl.getHost())) { - matchedShared.put(entry.getKey(), ru); - if (log.isDebugEnabled()) { - log.debug( - "Added Shared Broker - [{}] as possible " - + "Candidates for namespace - [{}] with policies", - brokerUrl.getHost(), namespace.toString()); - } - } else { - if (log.isDebugEnabled()) { - log.debug("Skipping Broker - [{}] not primary broker and not shared" - + " for namespace - [{}] ", brokerUrl.getHost(), namespace.toString()); - } + } + brokerCandidateCache.clear(); + try { + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, availableBrokersCache); + } catch (Exception e) { + log.warn("Error when trying to apply policies: {}", e); + for (final Map.Entry> entry : availableBrokers.entrySet()) { + result.putAll(entry.getKey(), entry.getValue()); + } + return result; + } - } - } else { - if (policies.isSharedBroker(brokerUrl.getHost())) { - matchedShared.put(entry.getKey(), ru); - log.debug("Added Shared Broker - [{}] as possible Candidates for namespace - [{}]", - brokerUrl.getHost(), namespace.toString()); + // After LoadManagerShared is finished applying the filter, put the results back into a multimap. + for (final Map.Entry> entry : availableBrokers.entrySet()) { + final Long rank = entry.getKey(); + final Set resourceUnits = entry.getValue(); + for (final ResourceUnit resourceUnit : resourceUnits) { + if (brokerCandidateCache.contains(resourceUnit.getResourceId().replace("http://", ""))) { + result.put(rank, resourceUnit); } } } - } - if (isIsolationPoliciesPresent) { - return getFinalCandidatesWithPolicy(namespace, matchedPrimaries, matchedShared); - } else { - log.debug( - "Policies not present for namespace - [{}] so only " - + "considering shared [{}] brokers for possible owner", - namespace.toString(), matchedShared.size()); - return getFinalCandidatesNoPolicy(matchedShared); + return result; } } @@ -951,7 +931,7 @@ private Map> getAvailableBrokers(ServiceUnitId serviceUn availableBrokers = Maps.newTreeMap(); for (String broker : activeBrokers) { ResourceUnit resourceUnit = new SimpleResourceUnit(String.format("http://%s", broker), - new PulsarResourceDescription()); + new PulsarResourceDescription()); availableBrokers.computeIfAbsent(0L, key -> Sets.newTreeSet()).add(resourceUnit); } log.info("Choosing at random from broker list: [{}]", availableBrokers.values()); @@ -960,7 +940,7 @@ private Map> getAvailableBrokers(ServiceUnitId serviceUn } private ResourceUnit getLeastLoadedBroker(ServiceUnitId serviceUnit, - Map> availableBrokers) { + Map> availableBrokers) { ResourceUnit selectedBroker = null; Multimap finalCandidates = getFinalCandidates(serviceUnit, availableBrokers); // Remove candidates that point to inactive brokers @@ -981,13 +961,14 @@ private ResourceUnit getLeastLoadedBroker(ServiceUnitId serviceUnit, } if (finalCandidates.size() > 0) { - if (this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LLS)) { + if (this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LLS) + || this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { selectedBroker = findBrokerForPlacement(finalCandidates, serviceUnit); } else { selectedBroker = placementStrategy.findBrokerForPlacement(finalCandidates); } - log.debug("Selected : [{}] for ServiceUnit : [{}]", selectedBroker.getResourceId(), - serviceUnit.getNamespaceObject().toString()); + log.info("Selected : [{}] for ServiceUnit : [{}]", selectedBroker.getResourceId(), + serviceUnit.getNamespaceObject().toString()); return selectedBroker; } else { // No available broker found @@ -1017,9 +998,9 @@ private void updateRanking() { try { String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker); LoadReport lr = loadReportCacheZk.get(key) - .orElseThrow(() -> new KeeperException.NoNodeException()); + .orElseThrow(() -> new KeeperException.NoNodeException()); ResourceUnit ru = new SimpleResourceUnit(String.format("http://%s", lr.getName()), - fromLoadReport(lr)); + fromLoadReport(lr)); this.currentLoadReports.put(ru, lr); } catch (Exception e) { log.warn("Error reading load report from Cache for broker - [{}], [{}]", broker, e); @@ -1035,14 +1016,14 @@ private void updateRanking() { public static boolean isAboveLoadLevel(SystemResourceUsage usage, float thresholdPercentage) { return (usage.bandwidthOut.percentUsage() > thresholdPercentage - || usage.bandwidthIn.percentUsage() > thresholdPercentage - || usage.cpu.percentUsage() > thresholdPercentage || usage.memory.percentUsage() > thresholdPercentage); + || usage.bandwidthIn.percentUsage() > thresholdPercentage + || usage.cpu.percentUsage() > thresholdPercentage || usage.memory.percentUsage() > thresholdPercentage); } public static boolean isBelowLoadLevel(SystemResourceUsage usage, float thresholdPercentage) { return (usage.bandwidthOut.percentUsage() < thresholdPercentage - && usage.bandwidthIn.percentUsage() < thresholdPercentage - && usage.cpu.percentUsage() < thresholdPercentage && usage.memory.percentUsage() < thresholdPercentage); + && usage.bandwidthIn.percentUsage() < thresholdPercentage + && usage.cpu.percentUsage() < thresholdPercentage && usage.memory.percentUsage() < thresholdPercentage); } private static long getRealtimeJvmHeapUsageMBytes() { @@ -1065,46 +1046,86 @@ private long getAverageJvmHeapUsageMBytes() { } private SystemResourceUsage getSystemResourceUsage() throws IOException { - SystemResourceUsage systemResourceUsage = brokerHostUsage.getBrokerHostUsage(); - - // Override System memory usage and limit with JVM heap usage and limit - long maxHeapMemoryInBytes = Runtime.getRuntime().maxMemory(); + SystemResourceUsage systemResourceUsage = LoadManagerShared.getSystemResourceUsage(brokerHostUsage); long memoryUsageInMBytes = getAverageJvmHeapUsageMBytes(); systemResourceUsage.memory.usage = (double) memoryUsageInMBytes; - systemResourceUsage.memory.limit = (double) (maxHeapMemoryInBytes) / MBytes; - - // Collect JVM direct memory - systemResourceUsage.directMemory.usage = (double) (sun.misc.SharedSecrets.getJavaNioAccess() - .getDirectBufferPool().getMemoryUsed() / MBytes); - systemResourceUsage.directMemory.limit = (double) (sun.misc.VM.maxDirectMemory() / MBytes); - return systemResourceUsage; } @Override public LoadReport generateLoadReport() throws Exception { - long timeSinceLastGenMillis = System.currentTimeMillis() - lastLoadReport.getTimestamp(); - if (timeSinceLastGenMillis <= LOAD_REPORT_UPDATE_MIMIMUM_INTERVAL) { - return lastLoadReport; - } + synchronized (bundleGainsCache) { + long timeSinceLastGenMillis = System.currentTimeMillis() - lastLoadReport.getTimestamp(); + if (timeSinceLastGenMillis <= LOAD_REPORT_UPDATE_MIMIMUM_INTERVAL) { + return lastLoadReport; + } + try { + LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + loadReport.setName(String.format("%s:%s", pulsar.getAdvertisedAddress(), + pulsar.getConfiguration().getWebServicePort())); + SystemResourceUsage systemResourceUsage = this.getSystemResourceUsage(); + loadReport.setOverLoaded(isAboveLoadLevel(systemResourceUsage, + this.getLoadBalancerBrokerOverloadedThresholdPercentage())); + loadReport.setUnderLoaded(isBelowLoadLevel(systemResourceUsage, + this.getLoadBalancerBrokerUnderloadedThresholdPercentage())); + + loadReport.setSystemResourceUsage(systemResourceUsage); + loadReport.setBundleStats(pulsar.getBrokerService().getBundleStats()); + loadReport.setTimestamp(System.currentTimeMillis()); + + final Set oldBundles = lastLoadReport.getBundles(); + final Set newBundles = loadReport.getBundles(); + bundleGainsCache.clear(); + bundleLossesCache.clear(); + + for (String oldBundle : oldBundles) { + if (!newBundles.contains(oldBundle)) { + bundleLossesCache.add(oldBundle); + } + } - try { - LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); - loadReport.setName(String.format("%s:%s", pulsar.getAdvertisedAddress(), pulsar.getConfiguration().getWebServicePort())); - SystemResourceUsage systemResourceUsage = this.getSystemResourceUsage(); - loadReport.setOverLoaded( - isAboveLoadLevel(systemResourceUsage, this.getLoadBalancerBrokerOverloadedThresholdPercentage())); - loadReport.setUnderLoaded( - isBelowLoadLevel(systemResourceUsage, this.getLoadBalancerBrokerUnderloadedThresholdPercentage())); - - loadReport.setSystemResourceUsage(systemResourceUsage); - loadReport.setBundleStats(pulsar.getBrokerService().getBundleStats()); - loadReport.setTimestamp(System.currentTimeMillis()); - return loadReport; - } catch (Exception e) { - log.error("[{}] Failed to generate LoadReport for broker, reason [{}]", e.getMessage(), e); - throw e; + for (String newBundle : newBundles) { + if (!oldBundles.contains(newBundle)) { + bundleGainsCache.add(newBundle); + } + } + loadReport.setBundleGains(bundleGainsCache); + loadReport.setBundleLosses(bundleLossesCache); + + final ResourceQuota allocatedQuota = getTotalAllocatedQuota(newBundles); + loadReport.setAllocatedCPU( + (allocatedQuota.getMsgRateIn() + allocatedQuota.getMsgRateOut()) * realtimeCpuLoadFactor); + loadReport.setAllocatedMemory(allocatedQuota.getMemory()); + loadReport.setAllocatedBandwidthIn(allocatedQuota.getBandwidthIn()); + loadReport.setAllocatedBandwidthOut(allocatedQuota.getBandwidthOut()); + loadReport.setAllocatedMsgRateIn(allocatedQuota.getMsgRateIn()); + loadReport.setAllocatedMsgRateOut(allocatedQuota.getMsgRateOut()); + + final ResourceUnit resourceUnit = new SimpleResourceUnit( + String.format("http://%s", loadReport.getName()), fromLoadReport(loadReport)); + Set preAllocatedBundles; + if (resourceUnitRankings.containsKey(resourceUnit)) { + preAllocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles(); + preAllocatedBundles.removeAll(newBundles); + } else { + preAllocatedBundles = new HashSet<>(); + } + + final ResourceQuota preAllocatedQuota = getTotalAllocatedQuota(preAllocatedBundles); + + loadReport.setPreAllocatedCPU( + (preAllocatedQuota.getMsgRateIn() + preAllocatedQuota.getMsgRateOut()) * realtimeCpuLoadFactor); + loadReport.setPreAllocatedMemory(preAllocatedQuota.getMemory()); + loadReport.setPreAllocatedBandwidthIn(preAllocatedQuota.getBandwidthIn()); + loadReport.setPreAllocatedBandwidthOut(preAllocatedQuota.getBandwidthOut()); + loadReport.setPreAllocatedMsgRateIn(preAllocatedQuota.getMsgRateIn()); + loadReport.setPreAllocatedMsgRateOut(preAllocatedQuota.getMsgRateOut()); + return loadReport; + } catch (Exception e) { + log.error("[{}] Failed to generate LoadReport for broker, reason [{}]", e.getMessage(), e); + throw e; + } } } @@ -1145,49 +1166,48 @@ public void writeLoadReportOnZookeeper() throws Exception { long newBundleCount = pulsar.getBrokerService().getNumberOfNamespaceBundles(); long bundleCountChange = Math.abs(oldBundleCount - newBundleCount); long maxCapacity = ResourceUnitRanking.calculateBrokerMaxCapacity( - lastLoadReport.getSystemResourceUsage(), - pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); + lastLoadReport.getSystemResourceUsage(), + pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); double bundlePercentageChange = (maxCapacity > 0) ? (bundleCountChange * 100 / maxCapacity) : 0; - if (newBundleCount < oldBundleCount || bundlePercentageChange > pulsar.getConfiguration() - .getLoadBalancerReportUpdateThresholdPercentage()) { + if (newBundleCount != oldBundleCount) { needUpdate = true; } // check resource usage comparing with last LoadReport if (!needUpdate && timestampNow - this.lastResourceUsageTimestamp > TimeUnit.MINUTES - .toMillis(pulsar.getConfiguration().getLoadBalancerHostUsageCheckIntervalMinutes())) { + .toMillis(pulsar.getConfiguration().getLoadBalancerHostUsageCheckIntervalMinutes())) { SystemResourceUsage oldUsage = lastLoadReport.getSystemResourceUsage(); SystemResourceUsage newUsage = this.getSystemResourceUsage(); this.lastResourceUsageTimestamp = timestampNow; // calculate percentage of change double cpuChange = (newUsage.cpu.limit > 0) - ? ((newUsage.cpu.usage - oldUsage.cpu.usage) * 100 / newUsage.cpu.limit) : 0; + ? ((newUsage.cpu.usage - oldUsage.cpu.usage) * 100 / newUsage.cpu.limit) : 0; double memChange = (newUsage.memory.limit > 0) - ? ((newUsage.memory.usage - oldUsage.memory.usage) * 100 / newUsage.memory.limit) : 0; + ? ((newUsage.memory.usage - oldUsage.memory.usage) * 100 / newUsage.memory.limit) : 0; double directMemChange = (newUsage.directMemory.limit > 0) - ? ((newUsage.directMemory.usage - oldUsage.directMemory.usage) * 100 - / newUsage.directMemory.limit) - : 0; + ? ((newUsage.directMemory.usage - oldUsage.directMemory.usage) * 100 + / newUsage.directMemory.limit) + : 0; double bandwidthOutChange = (newUsage.bandwidthOut.limit > 0) - ? ((newUsage.bandwidthOut.usage - oldUsage.bandwidthOut.usage) * 100 - / newUsage.bandwidthOut.limit) - : 0; + ? ((newUsage.bandwidthOut.usage - oldUsage.bandwidthOut.usage) * 100 + / newUsage.bandwidthOut.limit) + : 0; double bandwidthInChange = (newUsage.bandwidthIn.limit > 0) - ? ((newUsage.bandwidthIn.usage - oldUsage.bandwidthIn.usage) * 100 - / newUsage.bandwidthIn.limit) - : 0; + ? ((newUsage.bandwidthIn.usage - oldUsage.bandwidthIn.usage) * 100 + / newUsage.bandwidthIn.limit) + : 0; long resourceChange = (long) Math.min(100.0, - Math.max(Math.abs(cpuChange), - Math.max(Math.abs(directMemChange), Math.max(Math.abs(memChange), - Math.max(Math.abs(bandwidthOutChange), Math.abs(bandwidthInChange)))))); + Math.max(Math.abs(cpuChange), + Math.max(Math.abs(directMemChange), Math.max(Math.abs(memChange), + Math.max(Math.abs(bandwidthOutChange), Math.abs(bandwidthInChange)))))); if (resourceChange > pulsar.getConfiguration().getLoadBalancerReportUpdateThresholdPercentage()) { needUpdate = true; log.info("LoadReport update triggered by change on resource usage, detal ({}).", - String.format( - "cpu: %.1f%%, mem: %.1f%%, directMemory: %.1f%%, bandwidthIn: %.1f%%, bandwidthOut: %.1f%%)", - cpuChange, memChange, directMemChange, bandwidthInChange, bandwidthOutChange)); + String.format( + "cpu: %.1f%%, mem: %.1f%%, directMemory: %.1f%%, bandwidthIn: %.1f%%, bandwidthOut: %.1f%%)", + cpuChange, memChange, directMemChange, bandwidthInChange, bandwidthOutChange)); } } } @@ -1196,7 +1216,7 @@ public void writeLoadReportOnZookeeper() throws Exception { if (needUpdate) { LoadReport lr = generateLoadReport(); pulsar.getZkClient().setData(brokerZnodePath, ObjectMapperFactory.getThreadLocal().writeValueAsBytes(lr), - -1); + -1); this.lastLoadReport = lr; this.lastResourceUsageTimestamp = lr.getTimestamp(); // split-bundle if requires @@ -1204,24 +1224,9 @@ public void writeLoadReportOnZookeeper() throws Exception { } } - private String getNamespaceNameFromBundleName(String bundleName) { - // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF - int pos = bundleName.lastIndexOf("/"); - checkArgument(pos != -1); - return bundleName.substring(0, pos); - } - - private String getBundleRangeFromBundleName(String bundleName) { - // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF - int pos = bundleName.lastIndexOf("/"); - checkArgument(pos != -1); - return bundleName.substring(pos + 1, bundleName.length()); - } - // todo: changeme: this can be optimized, we don't have to iterate through everytime private boolean isBrokerAvailableForRebalancing(String bundleName, long maxLoadLevel) { - - NamespaceName namespaceName = new NamespaceName(getNamespaceNameFromBundleName(bundleName)); + NamespaceName namespaceName = new NamespaceName(LoadManagerShared.getNamespaceNameFromBundleName(bundleName)); Map> availableBrokers = sortedRankings.get(); // this does not have "http://" in front, hacky but no time to pretty up Multimap brokers = getFinalCandidates(namespaceName, availableBrokers); @@ -1250,10 +1255,10 @@ public boolean isUnloadDisabledInLoadShedding() { boolean unloadDisabledInLoadShedding = false; try { unloadDisabledInLoadShedding = pulsar.getGlobalZkCache() - .exists(AdminResource.LOAD_SHEDDING_UNLOAD_DISABLED_FLAG_PATH); + .exists(AdminResource.LOAD_SHEDDING_UNLOAD_DISABLED_FLAG_PATH); } catch (Exception e) { log.warn("Unable to fetch contents of [{}] from global zookeeper", - AdminResource.LOAD_SHEDDING_UNLOAD_DISABLED_FLAG_PATH, e); + AdminResource.LOAD_SHEDDING_UNLOAD_DISABLED_FLAG_PATH, e); } return unloadDisabledInLoadShedding; } @@ -1267,17 +1272,18 @@ private void unloadNamespacesFromOverLoadedBrokers(Map nam if (!isUnloadDisabledInLoadShedding()) { log.info("Unloading namespace {} from overloaded broker {}", bundleName, brokerName); adminCache.get(brokerName).namespaces().unloadNamespaceBundle( - getNamespaceNameFromBundleName(bundleName), getBundleRangeFromBundleName(bundleName)); + LoadManagerShared.getNamespaceNameFromBundleName(bundleName), + LoadManagerShared.getBundleRangeFromBundleName(bundleName)); log.info("Successfully unloaded namespace {} from broker {}", bundleName, brokerName); } else { log.info("DRY RUN: Unload in Load Shedding is disabled. Namespace {} would have been " - + "unloaded from overloaded broker {} otherwise.", bundleName, brokerName); + + "unloaded from overloaded broker {} otherwise.", bundleName, brokerName); } unloadedHotNamespaceCache.put(bundleName, System.currentTimeMillis()); } else { // we can't unload this namespace so move to next one log.info("Can't unload Namespace {} because it was unloaded last at {} and unload interval has " - + "not exceeded.", bundleName, LocalDateTime.now()); + + "not exceeded.", bundleName, LocalDateTime.now()); } } catch (Exception e) { log.warn("ERROR failed to unload the bundle {} from overloaded broker {}", bundleName, brokerName, e); @@ -1290,7 +1296,7 @@ public void doLoadShedding() { long overloadThreshold = this.getLoadBalancerBrokerOverloadedThresholdPercentage(); long comfortLoadLevel = this.getLoadBalancerBrokerComfortLoadThresholdPercentage(); log.info("Running load shedding task as leader broker, overload threshold {}, comfort loadlevel {}", - overloadThreshold, comfortLoadLevel); + overloadThreshold, comfortLoadLevel); // overloadedRU --> bundleName Map namespaceBundlesToBeUnloaded = new HashMap<>(); synchronized (currentLoadReports) { @@ -1305,9 +1311,9 @@ public void doLoadShedding() { // can't unload one namespace, just issue a warning message String bundleName = lr.getBundleStats().keySet().iterator().next(); log.warn( - "HIGH USAGE WARNING : Sole namespace bundle {} is overloading broker {}. " - + "No Load Shedding will be done on this broker", - bundleName, overloadedRU.getResourceId()); + "HIGH USAGE WARNING : Sole namespace bundle {} is overloading broker {}. " + + "No Load Shedding will be done on this broker", + bundleName, overloadedRU.getResourceId()); continue; } for (Map.Entry bundleStat : bundleStats.entrySet()) { @@ -1316,14 +1322,14 @@ public void doLoadShedding() { // We need at least one underloaded RU from list of candidates that can host this bundle if (isBrokerAvailableForRebalancing(bundleStat.getKey(), comfortLoadLevel)) { log.info( - "Namespace bundle {} will be unloaded from overloaded broker {}, bundle stats (topics: {}, producers {}, " - + "consumers {}, bandwidthIn {}, bandwidthOut {})", - bundleName, overloadedRU.getResourceId(), stats.topics, stats.producerCount, - stats.consumerCount, stats.msgThroughputIn, stats.msgThroughputOut); + "Namespace bundle {} will be unloaded from overloaded broker {}, bundle stats (topics: {}, producers {}, " + + "consumers {}, bandwidthIn {}, bandwidthOut {})", + bundleName, overloadedRU.getResourceId(), stats.topics, stats.producerCount, + stats.consumerCount, stats.msgThroughputIn, stats.msgThroughputOut); namespaceBundlesToBeUnloaded.put(overloadedRU, bundleName); } else { log.info("Unable to shed load from broker {}, no brokers with enough capacity available " - + "for re-balancing {}", overloadedRU.getResourceId(), bundleName); + + "for re-balancing {}", overloadedRU.getResourceId(), bundleName); } break; } @@ -1345,8 +1351,8 @@ public void doNamespaceBundleSplit() throws Exception { long maxBundleBandwidth = pulsar.getConfiguration().getLoadBalancerNamespaceBundleMaxBandwidthMbytes() * MBytes; log.info( - "Running namespace bundle split with thresholds: topics {}, sessions {}, msgRate {}, bandwidth {}, maxBundles {}", - maxBundleTopics, maxBundleSessions, maxBundleMsgRate, maxBundleBandwidth, maxBundleCount); + "Running namespace bundle split with thresholds: topics {}, sessions {}, msgRate {}, bandwidth {}, maxBundles {}", + maxBundleTopics, maxBundleSessions, maxBundleMsgRate, maxBundleBandwidth, maxBundleCount); if (this.lastLoadReport == null || this.lastLoadReport.getBundleStats() == null) { return; } @@ -1363,15 +1369,16 @@ public void doNamespaceBundleSplit() throws Exception { boolean needSplit = false; if (stats.topics > maxBundleTopics || totalSessions > maxBundleSessions || totalMsgRate > maxBundleMsgRate - || totalBandwidth > maxBundleBandwidth) { + || totalBandwidth > maxBundleBandwidth) { if (stats.topics <= 1) { log.info("Unable to split hot namespace bundle {} since there is only one topic.", bundleName); } else { - NamespaceName namespaceName = new NamespaceName(getNamespaceNameFromBundleName(bundleName)); + NamespaceName namespaceName = new NamespaceName( + LoadManagerShared.getNamespaceNameFromBundleName(bundleName)); int numBundles = pulsar.getNamespaceService().getBundleCount(namespaceName); if (numBundles >= maxBundleCount) { log.info("Unable to split hot namespace bundle {} since the namespace has too many bundles.", - bundleName); + bundleName); } else { needSplit = true; } @@ -1381,13 +1388,13 @@ public void doNamespaceBundleSplit() throws Exception { if (needSplit) { if (this.getLoadBalancerAutoBundleSplitEnabled()) { log.info( - "Will split hot namespace bundle {}, topics {}, producers+consumers {}, msgRate in+out {}, bandwidth in+out {}", - bundleName, stats.topics, totalSessions, totalMsgRate, totalBandwidth); + "Will split hot namespace bundle {}, topics {}, producers+consumers {}, msgRate in+out {}, bandwidth in+out {}", + bundleName, stats.topics, totalSessions, totalMsgRate, totalBandwidth); bundlesToBeSplit.add(bundleName); } else { log.info( - "DRY RUN - split hot namespace bundle {}, topics {}, producers+consumers {}, msgRate in+out {}, bandwidth in+out {}", - bundleName, stats.topics, totalSessions, totalMsgRate, totalBandwidth); + "DRY RUN - split hot namespace bundle {}, topics {}, producers+consumers {}, msgRate in+out {}, bandwidth in+out {}", + bundleName, stats.topics, totalSessions, totalMsgRate, totalBandwidth); } } } @@ -1396,7 +1403,8 @@ public void doNamespaceBundleSplit() throws Exception { for (String bundleName : bundlesToBeSplit) { try { pulsar.getAdminClient().namespaces().splitNamespaceBundle( - getNamespaceNameFromBundleName(bundleName), getBundleRangeFromBundleName(bundleName)); + LoadManagerShared.getNamespaceNameFromBundleName(bundleName), + LoadManagerShared.getBundleRangeFromBundleName(bundleName)); log.info("Successfully split namespace bundle {}", bundleName); } catch (Exception e) { log.error("Failed to split namespace bundle {}", bundleName, e); diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/NamespaceService.java index 31c77090d6304..c14ce263ec42e 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/NamespaceService.java @@ -31,10 +31,12 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; +import com.yahoo.pulsar.common.policies.data.loadbalancer.ServiceLookupData; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.AsyncCallback.StatCallback; @@ -93,7 +95,7 @@ public enum AddressType { private final ServiceConfiguration config; - private final LoadManager loadManager; + private final AtomicReference loadManager; private final PulsarService pulsar; @@ -148,12 +150,9 @@ private NamespaceBundle getFullBundle(NamespaceName fqnn) throws Exception { return bundleFactory.getFullBundle(fqnn); } - private static final Deserializer loadReportDeserializer = new Deserializer() { - @Override - public LoadReport deserialize(String key, byte[] content) throws Exception { - return jsonMapper().readValue(content, LoadReport.class); - } - }; + private static final Deserializer serviceLookupDataDeserializer = (key, content) -> + jsonMapper().readValue(content, ServiceLookupData.class); + public URL getWebServiceUrl(ServiceUnitId suName, boolean authoritative, boolean isRequestHttps, boolean readOnly) throws Exception { @@ -333,7 +332,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture< } if (candidateBroker == null) { - if (!this.loadManager.isCentralized() || pulsar.getLeaderElectionService().isLeader()) { + if (!this.loadManager.get().isCentralized() || pulsar.getLeaderElectionService().isLeader()) { candidateBroker = getLeastLoadedFromLoadManager(bundle); } else { if (authoritative) { @@ -406,13 +405,14 @@ private CompletableFuture createLookupResult(String candidateBroke try { checkArgument(StringUtils.isNotBlank(candidateBroker), "Lookup broker can't be null " + candidateBroker); URI uri = new URI(candidateBroker); - String path = String.format("%s/%s:%s", SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT, uri.getHost(), + String path = String.format("%s/%s:%s", LoadManager.LOADBALANCE_BROKERS_ROOT, uri.getHost(), uri.getPort()); - pulsar.getLocalZkCache().getDataAsync(path, loadReportDeserializer).thenAccept(reportData -> { + pulsar.getLocalZkCache().getDataAsync(path, serviceLookupDataDeserializer).thenAccept(reportData -> { if (reportData.isPresent()) { - LoadReport report = reportData.get(); - lookupFuture.complete(new LookupResult(report.getWebServiceUrl(), report.getWebServiceUrlTls(), - report.getPulsarServiceUrl(), report.getPulsarServieUrlTls())); + ServiceLookupData lookupData = reportData.get(); + lookupFuture.complete(new LookupResult(lookupData.getWebServiceUrl(), + lookupData.getWebServiceUrlTls(), lookupData.getPulsarServiceUrl(), + lookupData.getPulsarServiceUrlTls())); } else { lookupFuture.completeExceptionally(new KeeperException.NoNodeException(path)); } @@ -427,8 +427,7 @@ private CompletableFuture createLookupResult(String candidateBroke } private boolean isBrokerActive(String candidateBroker) throws KeeperException, InterruptedException { - Set activeNativeBrokers = pulsar.getLocalZkCache() - .getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT); + Set activeNativeBrokers = pulsar.getLocalZkCache().getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT); for (String brokerHostPort : activeNativeBrokers) { if (candidateBroker.equals("http://" + brokerHostPort)) { @@ -449,13 +448,11 @@ private boolean isBrokerActive(String candidateBroker) throws KeeperException, I /** * Helper function to encapsulate the logic to invoke between old and new load manager * - * @param namespaceName - * @param decidedByLeader * @return * @throws Exception */ private String getLeastLoadedFromLoadManager(ServiceUnitId serviceUnit) throws Exception { - String lookupAddress = loadManager.getLeastLoaded(serviceUnit).getResourceId(); + String lookupAddress = loadManager.get().getLeastLoaded(serviceUnit).getResourceId(); if (LOG.isDebugEnabled()) { LOG.debug("{} : redirecting to the least loaded broker, lookup address={}", pulsar.getWebServiceAddress(), lookupAddress); @@ -569,7 +566,7 @@ public CompletableFuture splitAndOwnBundle(NamespaceBundle bundle) throws bundleFactory.invalidateBundleCache(nsname); // update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(bundle); - loadManager.setLoadReportForceUpdateFlag(); + loadManager.get().setLoadReportForceUpdateFlag(); future.complete(null); } catch (Exception e) { String msg1 = format( diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/ServiceUnitZkUtils.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/ServiceUnitZkUtils.java index d80f5c53ba929..b11099de638e6 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/ServiceUnitZkUtils.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/namespace/ServiceUnitZkUtils.java @@ -116,7 +116,7 @@ private static final void cleanupNamespaceNodes(ZooKeeper zkc, String root, Stri // we don't need a watch here since we are only cleaning up the stale ephemeral nodes from previous session for (String node : zkc.getChildren(root, false)) { String currentPath = root + "/" + node; - // retrieve the content and try to decode with LookupData + // retrieve the content and try to decode with ServiceLookupData List children = zkc.getChildren(currentPath, false); if (children.size() == 0) { // clean up a single namespace node diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/BrokerService.java index ed856ab9eddd8..aebc181a3d33c 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/BrokerService.java @@ -40,6 +40,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import com.yahoo.pulsar.broker.loadbalance.LoadManager; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; import org.apache.bookkeeper.mledger.ManagedLedger; @@ -343,7 +344,7 @@ public void unloadNamespaceBundlesGracefully() { try { // make broker-node unavailable from the cluster if (pulsar.getLoadManager() != null) { - pulsar.getLoadManager().disableBroker(); + pulsar.getLoadManager().get().disableBroker(); } // unload all namespace-bundles gracefully @@ -910,6 +911,17 @@ private void updateConfigurationAndRegisterListeners() { // add listener on "maxConcurrentTopicLoadRequest" value change registerConfigurationListener("maxConcurrentTopicLoadRequest", (maxConcurrentTopicLoadRequest) -> topicLoadRequestSemaphore.set(new Semaphore((int) maxConcurrentTopicLoadRequest, false))); + registerConfigurationListener("loadManagerClassName", className -> { + try { + final LoadManager newLoadManager = LoadManager.create(pulsar); + log.info("Created load manager: {}", className); + pulsar.getLoadManager().get().disableBroker(); + newLoadManager.start(); + pulsar.getLoadManager().set(newLoadManager); + } catch (Exception ex) { + log.warn("Failed to change load manager due to {}", ex); + } + }); // add more listeners here } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/stats/MetricsGenerator.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/stats/MetricsGenerator.java index b37e79c49f2dd..ad0474b359f62 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/stats/MetricsGenerator.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/stats/MetricsGenerator.java @@ -48,7 +48,7 @@ private List collect() { metricsCollection.addAll(new ManagedLedgerCacheMetrics(pulsar).generate()); metricsCollection.addAll(new ManagedLedgerMetrics(pulsar).generate()); metricsCollection.addAll(pulsar.getBrokerService().getDestinationMetrics()); - metricsCollection.addAll(pulsar.getLoadManager().getLoadBalancingMetrics()); + metricsCollection.addAll(pulsar.getLoadManager().get().getLoadBalancingMetrics()); return metricsCollection; } diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/LoadBalancerTest.java index 45f591c7e6242..52b6924fe6081 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -203,11 +203,11 @@ public void testLoadReportsWrittenOnZK() throws Exception { assertFalse(loadReport.isOverLoaded()); // Check Initial Ranking is populated in both the brokers - Field ranking = ((SimpleLoadManagerImpl) pulsarServices[i].getLoadManager()).getClass() + Field ranking = ((SimpleLoadManagerImpl) pulsarServices[i].getLoadManager().get()).getClass() .getDeclaredField("sortedRankings"); ranking.setAccessible(true); AtomicReference>> sortedRanking = (AtomicReference>>) ranking - .get(pulsarServices[i].getLoadManager()); + .get(pulsarServices[i].getLoadManager().get()); printSortedRanking(sortedRanking); // all brokers have same rank to it would be 0 --> set-of-all-the-brokers @@ -217,7 +217,7 @@ public void testLoadReportsWrittenOnZK() throws Exception { } assertEquals(brokerCount, BROKER_COUNT); DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns/test-topic"); - ResourceUnit found = pulsarServices[i].getLoadManager() + ResourceUnit found = pulsarServices[i].getLoadManager().get() .getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(fqdn)); assertTrue(found != null); } @@ -256,7 +256,7 @@ public void testUpdateLoadReportAndCheckUpdatedRanking() throws Exception { Map namespaceOwner = new HashMap<>(); for (int i = 0; i < totalNamespaces; i++) { DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic"); - ResourceUnit found = pulsarServices[0].getLoadManager() + ResourceUnit found = pulsarServices[0].getLoadManager().get() .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)); if (namespaceOwner.containsKey(found.getResourceId())) { namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1); @@ -279,10 +279,10 @@ public void testUpdateLoadReportAndCheckUpdatedRanking() throws Exception { private AtomicReference>> getSortedRanking(PulsarService pulsar) throws NoSuchFieldException, IllegalAccessException { - Field ranking = ((SimpleLoadManagerImpl) pulsar.getLoadManager()).getClass().getDeclaredField("sortedRankings"); + Field ranking = ((SimpleLoadManagerImpl) pulsar.getLoadManager().get()).getClass().getDeclaredField("sortedRankings"); ranking.setAccessible(true); AtomicReference>> sortedRanking = (AtomicReference>>) ranking - .get(pulsar.getLoadManager()); + .get(pulsar.getLoadManager().get()); return sortedRanking; } @@ -395,7 +395,7 @@ public void testDestinationAssignmentWithExistingBundles() throws Exception { Map namespaceOwner = new HashMap<>(); for (int i = 0; i < totalNamespaces; i++) { DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic"); - ResourceUnit found = pulsarServices[0].getLoadManager() + ResourceUnit found = pulsarServices[0].getLoadManager().get() .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)); if (namespaceOwner.containsKey(found.getResourceId())) { namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1); @@ -422,11 +422,11 @@ public void testDestinationAssignmentWithExistingBundles() throws Exception { private AtomicReference> getRealtimeResourceQuota(PulsarService pulsar) throws NoSuchFieldException, IllegalAccessException { - Field quotasField = ((SimpleLoadManagerImpl) pulsar.getLoadManager()).getClass() + Field quotasField = ((SimpleLoadManagerImpl) pulsar.getLoadManager().get()).getClass() .getDeclaredField("realtimeResourceQuotas"); quotasField.setAccessible(true); AtomicReference> realtimeResourceQuotas = (AtomicReference>) quotasField - .get(pulsar.getLoadManager()); + .get(pulsar.getLoadManager().get()); return realtimeResourceQuotas; } @@ -637,14 +637,14 @@ public void testNamespaceBundleAutoSplit() throws Exception { newBundleStats(maxTopics + 1, 0, 0, 0, 0, 0, 0)); lr.setBundleStats(bundleStats); - setObjectField(SimpleLoadManagerImpl.class, pulsarServices[0].getLoadManager(), "lastLoadReport", lr); + setObjectField(SimpleLoadManagerImpl.class, pulsarServices[0].getLoadManager().get(), "lastLoadReport", lr); String znodePath = String.format("%s/%s", SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT, lookupAddresses[0]); String loadReportJson = objectMapper.writeValueAsString(lr); bkEnsemble.getZkClient().setData(znodePath, loadReportJson.getBytes(Charsets.UTF_8), -1); // sleep to wait load ranking be triggered and trigger bundle split Thread.sleep(5000); - pulsarServices[0].getLoadManager().doNamespaceBundleSplit(); + pulsarServices[0].getLoadManager().get().doNamespaceBundleSplit(); // verify bundles are split verify(namespaceAdmin, times(1)).splitNamespaceBundle("pulsar/use/primary-ns-01", "0x00000000_0x80000000"); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index 7d80959408323..a3c33f091c517 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -452,11 +452,13 @@ public void testBrokerHostUsage() { @Test public void testTask() throws Exception { LoadManager loadManager = mock(LoadManager.class); - LoadResourceQuotaUpdaterTask task1 = new LoadResourceQuotaUpdaterTask(loadManager); + AtomicReference atomicLoadManager = new AtomicReference<>(loadManager); + LoadResourceQuotaUpdaterTask task1 = new LoadResourceQuotaUpdaterTask(atomicLoadManager); task1.run(); verify(loadManager, times(1)).writeResourceQuotasToZooKeeper(); - LoadSheddingTask task2 = new LoadSheddingTask(loadManager); + + LoadSheddingTask task2 = new LoadSheddingTask(atomicLoadManager); task2.run(); verify(loadManager, times(1)).doLoadShedding(); } diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/BrokerServiceLookupTest.java index 632df94cdaa44..573045003c270 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/BrokerServiceLookupTest.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import javax.naming.AuthenticationException; import javax.net.ssl.HttpsURLConnection; @@ -115,24 +116,24 @@ public void testMultipleBrokerLookup() throws Exception { conf2.setAdvertisedAddress("localhost"); conf2.setClusterName(conf.getClusterName()); PulsarService pulsar2 = startBroker(conf2); - pulsar.getLoadManager().writeLoadReportOnZookeeper(); - pulsar2.getLoadManager().writeLoadReportOnZookeeper(); + pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); + pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager()); + LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); + LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: redirect request to leader [2] doReturn(true).when(loadManager2).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); // mock: return Broker2 as a Least-loaded broker when leader receies request [3] doReturn(true).when(loadManager1).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); /**** started broker-2 ****/ @@ -202,8 +203,8 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { PulsarService pulsar2 = startBroker(conf2); - pulsar.getLoadManager().writeLoadReportOnZookeeper(); - pulsar2.getLoadManager().writeLoadReportOnZookeeper(); + pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); + pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); URI brokerServiceUrl = new URI(broker2ServiceUrl); PulsarClient pulsarClient2 = PulsarClient.create(brokerServiceUrl.toString(), new ClientConfiguration()); @@ -214,7 +215,7 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { stopBroker(); startBroker(); - LoadManager loadManager2 = spy(pulsar2.getLoadManager()); + LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); @@ -222,7 +223,7 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { doReturn(true).when(loadManager2).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); doReturn(resourceUnit).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager2); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager2)); /**** started broker-2 ****/ // load namespace-bundle by calling Broker2 @@ -286,22 +287,22 @@ public void testPartitionTopicLookup() throws Exception { conf2.setAdvertisedAddress("localhost"); conf2.setClusterName(pulsar.getConfiguration().getClusterName()); PulsarService pulsar2 = startBroker(conf2); - pulsar.getLoadManager().writeLoadReportOnZookeeper(); - pulsar2.getLoadManager().writeLoadReportOnZookeeper(); + pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); + pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager()); + LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); + LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: return Broker2 as a Least-loaded broker when leader receies request doReturn(true).when(loadManager1).isCentralized(); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); // mock: redirect request to leader doReturn(true).when(loadManager2).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); /**** broker-2 started ****/ ProducerConfiguration producerConf = new ProducerConfiguration(); @@ -372,24 +373,24 @@ public void testWebserviceServiceTls() throws Exception { conf.setTlsKeyFilePath(TLS_SERVER_KEY_FILE_PATH); stopBroker(); startBroker(); - pulsar.getLoadManager().writeLoadReportOnZookeeper(); - pulsar2.getLoadManager().writeLoadReportOnZookeeper(); + pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); + pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager()); + LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); + LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: redirect request to leader [2] doReturn(true).when(loadManager2).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); // mock: return Broker2 as a Least-loaded broker when leader receies // request [3] doReturn(true).when(loadManager1).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); /**** started broker-2 ****/ diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java index e5b939966789d..8401524e6c345 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -164,7 +164,7 @@ public void testDisconnectClientWithoutClosingConnection() throws Exception { consumers.put(0, consumer1); // disable this broker to avoid any new requests - pulsar.getLoadManager().disableBroker(); + pulsar.getLoadManager().get().disableBroker(); NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn1)); NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn2)); diff --git a/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/LoadReport.java b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/LoadReport.java index 2cc4a7cc8104d..4209845255e2d 100644 --- a/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/LoadReport.java +++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/LoadReport.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.Maps; + import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage.ResourceType; @@ -30,15 +31,13 @@ * This class represents the overall load of the broker - it includes overall {@link SystemResourceUsage} and * {@link NamespaceUsage} for all the namespaces hosted by this broker. */ -public class LoadReport { +public class LoadReport implements ServiceLookupData { private String name; private final String webServiceUrl; private final String webServiceUrlTls; - private final String pulsarServiceUrl; - private final String pulsarServieUrlTls; - + private final String pulsarServiceUrlTls; private boolean isUnderLoaded; private boolean isOverLoaded; private long timestamp; @@ -53,12 +52,13 @@ public LoadReport() { this(null, null, null, null); } - public LoadReport(String webServiceUrl, String webServiceUrlTls, String pulsarServiceUrl, String pulsarServieUrlTls) { - this.webServiceUrl = webServiceUrl; + public LoadReport(String webServiceUrl, String webServiceUrlTls, String pulsarServiceUrl, String pulsarServiceUrlTls) { + this.webServiceUrl = webServiceUrl; this.webServiceUrlTls = webServiceUrlTls; this.pulsarServiceUrl = pulsarServiceUrl; - this.pulsarServieUrlTls = pulsarServieUrlTls; - + this.pulsarServiceUrlTls = pulsarServiceUrlTls; + bundleLosses = new HashSet<>(); + bundleGains = new HashSet<>(); isUnderLoaded = false; isOverLoaded = false; timestamp = 0; @@ -77,6 +77,24 @@ public LoadReport(String webServiceUrl, String webServiceUrlTls, String pulsarS private Map bundleStats; + private Set bundleGains; + + private Set bundleLosses; + + private double allocatedCPU; + private double allocatedMemory; + private double allocatedBandwidthIn; + private double allocatedBandwidthOut; + private double allocatedMsgRateIn; + private double allocatedMsgRateOut; + + private double preAllocatedCPU; + private double preAllocatedMemory; + private double preAllocatedBandwidthIn; + private double preAllocatedBandwidthOut; + private double preAllocatedMsgRateIn; + private double preAllocatedMsgRateOut; + public void setBundleStats(Map stats) { bundleStats = (stats == null) ? null : new HashMap(stats); } @@ -226,19 +244,135 @@ public TreeMap getSortedBundleStats(ResourceType r return sortedBundleStats; } + public Set getBundleGains() { + return bundleGains; + } + + public void setBundleGains(Set bundleGains) { + this.bundleGains = bundleGains; + } + + public Set getBundleLosses() { + return bundleLosses; + } + + public void setBundleLosses(Set bundleLosses) { + this.bundleLosses = bundleLosses; + } + + public double getAllocatedCPU() { + return allocatedCPU; + } + + public void setAllocatedCPU(double allocatedCPU) { + this.allocatedCPU = allocatedCPU; + } + + public double getAllocatedMemory() { + return allocatedMemory; + } + + public void setAllocatedMemory(double allocatedMemory) { + this.allocatedMemory = allocatedMemory; + } + + public double getAllocatedBandwidthIn() { + return allocatedBandwidthIn; + } + + public void setAllocatedBandwidthIn(double allocatedBandwidthIn) { + this.allocatedBandwidthIn = allocatedBandwidthIn; + } + + public double getAllocatedBandwidthOut() { + return allocatedBandwidthOut; + } + + public void setAllocatedBandwidthOut(double allocatedBandwidthOut) { + this.allocatedBandwidthOut = allocatedBandwidthOut; + } + + public double getAllocatedMsgRateIn() { + return allocatedMsgRateIn; + } + + public void setAllocatedMsgRateIn(double allocatedMsgRateIn) { + this.allocatedMsgRateIn = allocatedMsgRateIn; + } + + public double getAllocatedMsgRateOut() { + return allocatedMsgRateOut; + } + + public void setAllocatedMsgRateOut(double allocatedMsgRateOut) { + this.allocatedMsgRateOut = allocatedMsgRateOut; + } + + public double getPreAllocatedCPU() { + return preAllocatedCPU; + } + + public void setPreAllocatedCPU(double preAllocatedCPU) { + this.preAllocatedCPU = preAllocatedCPU; + } + + public double getPreAllocatedMemory() { + return preAllocatedMemory; + } + + public void setPreAllocatedMemory(double preAllocatedMemory) { + this.preAllocatedMemory = preAllocatedMemory; + } + + public double getPreAllocatedBandwidthIn() { + return preAllocatedBandwidthIn; + } + + public void setPreAllocatedBandwidthIn(double preAllocatedBandwidthIn) { + this.preAllocatedBandwidthIn = preAllocatedBandwidthIn; + } + + public double getPreAllocatedBandwidthOut() { + return preAllocatedBandwidthOut; + } + + public void setPreAllocatedBandwidthOut(double preAllocatedBandwidthOut) { + this.preAllocatedBandwidthOut = preAllocatedBandwidthOut; + } + + public double getPreAllocatedMsgRateIn() { + return preAllocatedMsgRateIn; + } + + public void setPreAllocatedMsgRateIn(double preAllocatedMsgRateIn) { + this.preAllocatedMsgRateIn = preAllocatedMsgRateIn; + } + + public double getPreAllocatedMsgRateOut() { + return preAllocatedMsgRateOut; + } + + public void setPreAllocatedMsgRateOut(double preAllocatedMsgRateOut) { + this.preAllocatedMsgRateOut = preAllocatedMsgRateOut; + } + + @Override public String getWebServiceUrl() { return webServiceUrl; } + @Override public String getWebServiceUrlTls() { return webServiceUrlTls; } + @Override public String getPulsarServiceUrl() { return pulsarServiceUrl; } - public String getPulsarServieUrlTls() { - return pulsarServieUrlTls; + @Override + public String getPulsarServiceUrlTls() { + return pulsarServiceUrlTls; } } diff --git a/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ResourceUnitRanking.java b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ResourceUnitRanking.java index d397e8378fb15..0ea502de7a00b 100644 --- a/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ResourceUnitRanking.java +++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ResourceUnitRanking.java @@ -39,6 +39,9 @@ public class ResourceUnitRanking implements Comparable { // estimated percentage of resource usage with the already assigned (both loaded and to-be-loaded) bundles private double estimatedLoadPercentage; + // estimated number of total messages with the already assigned (both loaded and to-be-loaded) bundles + private double estimatedMessageRate; + private double allocatedLoadPercentageCPU; private double allocatedLoadPercentageMemory; private double allocatedLoadPercentageBandwidthIn; @@ -113,6 +116,10 @@ private void estimateLoadPercentage() { this.estimatedLoadPercentage = Math.max(this.estimatedLoadPercentageCPU, Math.max(this.estimatedLoadPercentageMemory, Math.max(this.estimatedLoadPercentageDirectMemory, Math.max(this.estimatedLoadPercentageBandwidthIn, this.estimatedLoadPercentageBandwidthOut)))); + + this.estimatedMessageRate = this.allocatedQuota.getMsgRateIn() + this.allocatedQuota.getMsgRateOut() + + this.preAllocatedQuota.getMsgRateIn() + this.preAllocatedQuota.getMsgRateOut(); + } public int compareTo(ResourceUnitRanking other) { @@ -138,6 +145,13 @@ public int compareTo(ResourceUnitRanking other) { return Double.compare(this.estimatedLoadPercentage, other.estimatedLoadPercentage); } + /** + * Compare two loads based on message rate only + */ + public int compareMessageRateTo(ResourceUnitRanking other) { + return Double.compare(this.estimatedMessageRate, other.estimatedMessageRate); + } + /** * If the ResourceUnit is idle */ @@ -192,6 +206,13 @@ public double getEstimatedLoadPercentage() { return this.estimatedLoadPercentage; } + /** + * Get the estimated message rate + */ + public double getEstimatedMessageRate() { + return this.estimatedMessageRate; + } + /** * Percentage of CPU allocated to bundle's quota */ @@ -225,7 +246,8 @@ public double getAllocatedLoadPercentageBandwidthOut() { */ public String getEstimatedLoadPercentageString() { return String.format( - "load: %.1f%% - cpu: %.1f%%, mem: %.1f%%, directMemory: %.1f%%, bandwidthIn: %.1f%%, bandwidthOut: %.1f%%", + "msgrate: %.0f, load: %.1f%% - cpu: %.1f%%, mem: %.1f%%, directMemory: %.1f%%, bandwidthIn: %.1f%%, bandwidthOut: %.1f%%", + this.estimatedMessageRate, this.estimatedLoadPercentage, this.estimatedLoadPercentageCPU, this.estimatedLoadPercentageMemory, this.estimatedLoadPercentageDirectMemory, this.estimatedLoadPercentageBandwidthIn, this.estimatedLoadPercentageBandwidthOut); diff --git a/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ServiceLookupData.java b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ServiceLookupData.java new file mode 100644 index 0000000000000..5940f53ec1fec --- /dev/null +++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ServiceLookupData.java @@ -0,0 +1,27 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.common.policies.data.loadbalancer; + +// For backwards compatibility purposes. +public interface ServiceLookupData { + public String getWebServiceUrl(); + + public String getWebServiceUrlTls(); + + public String getPulsarServiceUrl(); + + public String getPulsarServiceUrlTls(); +} diff --git a/pulsar-discovery-service/src/main/java/com/yahoo/pulsar/discovery/service/ServerConnection.java b/pulsar-discovery-service/src/main/java/com/yahoo/pulsar/discovery/service/ServerConnection.java index 9fb2c4c320e4a..ef5e050e2ce6e 100644 --- a/pulsar-discovery-service/src/main/java/com/yahoo/pulsar/discovery/service/ServerConnection.java +++ b/pulsar-discovery-service/src/main/java/com/yahoo/pulsar/discovery/service/ServerConnection.java @@ -129,7 +129,7 @@ private void sendLookupResponse(long requestId) { try { LoadReport availableBroker = service.getDiscoveryProvider().nextBroker(); ctx.writeAndFlush(Commands.newLookupResponse(availableBroker.getPulsarServiceUrl(), - availableBroker.getPulsarServieUrlTls(), false, Redirect, requestId)); + availableBroker.getPulsarServiceUrlTls(), false, Redirect, requestId)); } catch (PulsarServerException e) { LOG.warn("[{}] Failed to get next active broker {}", remoteAddress, e.getMessage(), e); ctx.writeAndFlush( diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 99b5ab3862f3a..c6b03f4575600 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -38,11 +38,25 @@ test-jar test + + org.apache.zookeeper + zookeeper + + + ${project.groupId} + pulsar-client-admin + ${project.version} + ${project.groupId} pulsar-client ${project.version} + + ${project.groupId} + pulsar-broker + ${project.version} + commons-configuration commons-configuration diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java new file mode 100644 index 0000000000000..dae3492be97a5 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java @@ -0,0 +1,374 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.testclient; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +import org.apache.commons.lang.SystemUtils; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import com.google.common.util.concurrent.RateLimiter; +import com.yahoo.pulsar.client.api.ClientConfiguration; +import com.yahoo.pulsar.client.api.Consumer; +import com.yahoo.pulsar.client.api.ConsumerConfiguration; +import com.yahoo.pulsar.client.api.MessageId; +import com.yahoo.pulsar.client.api.MessageListener; +import com.yahoo.pulsar.client.api.Producer; +import com.yahoo.pulsar.client.api.ProducerConfiguration; +import com.yahoo.pulsar.client.api.PulsarClient; +import com.yahoo.pulsar.client.impl.PulsarClientImpl; + +import io.netty.channel.EventLoopGroup; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.util.concurrent.DefaultThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LoadSimulationClient is used to simulate client load by maintaining producers and consumers for topics. Instances of + * this class are controlled across a network via LoadSimulationController. + */ +public class LoadSimulationClient { + private final static Logger log = LoggerFactory.getLogger(LoadSimulationClient.class); + + // Values for command responses. + public static final byte FOUND_TOPIC = 0; + public static final byte NO_SUCH_TOPIC = 1; + public static final byte REDUNDANT_COMMAND = 2; + + // Values for command encodings. + public static final byte CHANGE_COMMAND = 0; + public static final byte STOP_COMMAND = 1; + public static final byte TRADE_COMMAND = 2; + public static final byte CHANGE_GROUP_COMMAND = 3; + public static final byte STOP_GROUP_COMMAND = 4; + + private final ExecutorService executor; + private final Map payloadCache; + private final Map topicsToTradeUnits; + private final PulsarClient client; + private final ProducerConfiguration producerConf; + private final ConsumerConfiguration consumerConf; + private final ClientConfiguration clientConf; + private final int port; + + // A TradeUnit is a Consumer and Producer pair. The rate of message + // consumption as well as size may be changed at + // any time, and the TradeUnit may also be stopped. + private static class TradeUnit { + Future producerFuture; + Future consumerFuture; + final AtomicBoolean stop; + final RateLimiter rateLimiter; + + // Creating a byte[] for every message is stressful for a client + // machine, so in order to ensure that any + // message size may be sent/changed while reducing object creation, the + // byte[] is wrapped in an AtomicReference. + final AtomicReference payload; + final ProducerConfiguration producerConf; + final PulsarClient client; + final String topic; + final Map payloadCache; + + public TradeUnit(final TradeConfiguration tradeConf, final PulsarClient client, + final ProducerConfiguration producerConf, final ConsumerConfiguration consumerConf, + final Map payloadCache) throws Exception { + consumerFuture = client.subscribeAsync(tradeConf.topic, "Subscriber-" + tradeConf.topic, consumerConf); + producerFuture = client.createProducerAsync(tradeConf.topic, producerConf); + this.payload = new AtomicReference<>(); + this.producerConf = producerConf; + this.payloadCache = payloadCache; + this.client = client; + topic = tradeConf.topic; + + // Add a byte[] of the appropriate size if it is not already present + // in the cache. + this.payload.set(payloadCache.computeIfAbsent(tradeConf.size, byte[]::new)); + rateLimiter = RateLimiter.create(tradeConf.rate); + stop = new AtomicBoolean(false); + } + + // Change the message rate/size according to the given configuration. + public void change(final TradeConfiguration tradeConf) { + rateLimiter.setRate(tradeConf.rate); + this.payload.set(payloadCache.computeIfAbsent(tradeConf.size, byte[]::new)); + } + + // Attempt to create a Producer indefinitely. Useful for ensuring + // messages continue to be sent after broker + // restarts occur. + private Producer getNewProducer() throws Exception { + while (true) { + try { + return client.createProducer(topic, producerConf); + } catch (Exception e) { + Thread.sleep(10000); + } + } + } + + private class MutableBoolean { + public volatile boolean value = true; + } + + public void start() throws Exception { + Producer producer = producerFuture.get(); + final Consumer consumer = consumerFuture.get(); + while (!stop.get()) { + final MutableBoolean wellnessFlag = new MutableBoolean(); + final Function exceptionHandler = e -> { + // Unset the well flag in the case of an exception so we can + // try to get a new Producer. + wellnessFlag.value = false; + return null; + }; + while (!stop.get() && wellnessFlag.value) { + producer.sendAsync(payload.get()).exceptionally(exceptionHandler); + rateLimiter.acquire(); + } + producer.closeAsync(); + if (!stop.get()) { + // The Producer failed due to an exception: attempt to get + // another producer. + producer = getNewProducer(); + } else { + // We are finished: close the consumer. + consumer.closeAsync(); + } + } + } + } + + // JCommander arguments for starting a LoadSimulationClient. + private static class MainArguments { + @Parameter(names = { "-h", "--help" }, description = "Help message", help = true) + boolean help; + + @Parameter(names = { "--port" }, description = "Port to listen on for controller", required = true) + public int port; + + @Parameter(names = { "--service-url" }, description = "Pulsar Service URL", required = true) + public String serviceURL; + } + + // Configuration class for initializing or modifying TradeUnits. + private static class TradeConfiguration { + public byte command; + public String topic; + public double rate; + public int size; + public String tenant; + public String group; + + public TradeConfiguration() { + command = -1; + rate = 100; + size = 1024; + } + } + + // Handle input sent from a controller. + private void handle(final Socket socket) throws Exception { + final DataInputStream inputStream = new DataInputStream(socket.getInputStream()); + int command; + while ((command = inputStream.read()) != -1) { + handle((byte) command, inputStream, new DataOutputStream(socket.getOutputStream())); + } + } + + // Decode TradeConfiguration fields common for topic creation and + // modification. + private void decodeProducerOptions(final TradeConfiguration tradeConf, final DataInputStream inputStream) + throws Exception { + tradeConf.topic = inputStream.readUTF(); + tradeConf.size = inputStream.readInt(); + tradeConf.rate = inputStream.readDouble(); + } + + // Decode TradeConfiguration fields common for group commands. + private void decodeGroupOptions(final TradeConfiguration tradeConf, final DataInputStream inputStream) + throws Exception { + tradeConf.tenant = inputStream.readUTF(); + tradeConf.group = inputStream.readUTF(); + } + + // Handle a command sent from a controller. + private void handle(final byte command, final DataInputStream inputStream, final DataOutputStream outputStream) + throws Exception { + final TradeConfiguration tradeConf = new TradeConfiguration(); + tradeConf.command = command; + switch (command) { + case CHANGE_COMMAND: + // Change the topic's settings if it exists. Report whether the + // topic was found on this server. + decodeProducerOptions(tradeConf, inputStream); + if (topicsToTradeUnits.containsKey(tradeConf.topic)) { + topicsToTradeUnits.get(tradeConf.topic).change(tradeConf); + outputStream.write(FOUND_TOPIC); + } else { + outputStream.write(NO_SUCH_TOPIC); + } + break; + case STOP_COMMAND: + // Stop the topic if it exists. Report whether the topic was found, + // and whether it was already stopped. + tradeConf.topic = inputStream.readUTF(); + if (topicsToTradeUnits.containsKey(tradeConf.topic)) { + final boolean wasStopped = topicsToTradeUnits.get(tradeConf.topic).stop.getAndSet(true); + outputStream.write(wasStopped ? REDUNDANT_COMMAND : FOUND_TOPIC); + } else { + outputStream.write(NO_SUCH_TOPIC); + } + break; + case TRADE_COMMAND: + // Create the topic. It is assumed that the topic does not already + // exist. + decodeProducerOptions(tradeConf, inputStream); + final TradeUnit tradeUnit = new TradeUnit(tradeConf, client, producerConf, consumerConf, payloadCache); + topicsToTradeUnits.put(tradeConf.topic, tradeUnit); + executor.submit(() -> { + try { + tradeUnit.start(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + // Tell controller topic creation is finished. + outputStream.write(NO_SUCH_TOPIC); + break; + case CHANGE_GROUP_COMMAND: + // Change the settings of all topics belonging to a group. Report + // the number of topics changed. + decodeGroupOptions(tradeConf, inputStream); + tradeConf.size = inputStream.readInt(); + tradeConf.rate = inputStream.readDouble(); + // See if a topic belongs to this tenant and group using this regex. + final String groupRegex = ".*://.*/" + tradeConf.tenant + "/" + tradeConf.group + "-.*/.*"; + int numFound = 0; + for (Map.Entry entry : topicsToTradeUnits.entrySet()) { + final String destination = entry.getKey(); + final TradeUnit unit = entry.getValue(); + if (destination.matches(groupRegex)) { + ++numFound; + unit.change(tradeConf); + } + } + outputStream.writeInt(numFound); + break; + case STOP_GROUP_COMMAND: + // Stop all topics belonging to a group. Report the number of topics + // stopped. + decodeGroupOptions(tradeConf, inputStream); + // See if a topic belongs to this tenant and group using this regex. + final String regex = ".*://.*/" + tradeConf.tenant + "/" + tradeConf.group + "-.*/.*"; + int numStopped = 0; + for (Map.Entry entry : topicsToTradeUnits.entrySet()) { + final String destination = entry.getKey(); + final TradeUnit unit = entry.getValue(); + if (destination.matches(regex) && !unit.stop.getAndSet(true)) { + ++numStopped; + } + } + outputStream.writeInt(numStopped); + break; + default: + throw new IllegalArgumentException("Unrecognized command code received: " + command); + } + outputStream.flush(); + } + + private static final MessageListener ackListener = Consumer::acknowledgeAsync; + + public LoadSimulationClient(final MainArguments arguments) throws Exception { + payloadCache = new ConcurrentHashMap<>(); + topicsToTradeUnits = new ConcurrentHashMap<>(); + final EventLoopGroup eventLoopGroup = SystemUtils.IS_OS_LINUX + ? new EpollEventLoopGroup(Runtime.getRuntime().availableProcessors(), + new DefaultThreadFactory("pulsar-test-client")) + : new NioEventLoopGroup(Runtime.getRuntime().availableProcessors(), + new DefaultThreadFactory("pulsar-test-client")); + clientConf = new ClientConfiguration(); + + clientConf.setConnectionsPerBroker(4); + + // Disable stats on the clients to reduce CPU/memory usage. + clientConf.setStatsInterval(0, TimeUnit.SECONDS); + + producerConf = new ProducerConfiguration(); + + // Disable timeout. + producerConf.setSendTimeout(0, TimeUnit.SECONDS); + + producerConf.setMessageRoutingMode(ProducerConfiguration.MessageRoutingMode.RoundRobinPartition); + + // Enable batching. + producerConf.setBatchingMaxPublishDelay(1, TimeUnit.MILLISECONDS); + producerConf.setBatchingEnabled(true); + consumerConf = new ConsumerConfiguration(); + consumerConf.setMessageListener(ackListener); + client = new PulsarClientImpl(arguments.serviceURL, clientConf, eventLoopGroup); + port = arguments.port; + executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); + } + + public static void main(String[] args) throws Exception { + final MainArguments mainArguments = new MainArguments(); + final JCommander jc = new JCommander(mainArguments); + try { + jc.parse(args); + } catch (ParameterException e) { + jc.usage(); + throw e; + } + (new LoadSimulationClient(mainArguments)).run(); + } + + public void run() throws Exception { + final ServerSocket serverSocket = new ServerSocket(port); + + while (true) { + // Technically, two controllers can be connected simultaneously, but + // non-sequential handling of commands + // has not been tested or considered and is not recommended. + log.info("Listening for controller command..."); + final Socket socket = serverSocket.accept(); + log.info("Connected to {}\n", socket.getInetAddress().getHostName()); + executor.submit(() -> { + try { + handle(socket); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + } + } +} diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java new file mode 100644 index 0000000000000..4786009431518 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -0,0 +1,661 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.testclient; + +import java.io.BufferedReader; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.net.Socket; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; + +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.common.policies.data.ResourceQuota; +import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; +import com.yahoo.pulsar.common.util.ObjectMapperFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * To use: 1. Delegate a list of server machines which act as zookeeper clients. 2. Choose a port for those machines. 3. + * On each of these machines, get them to listen via pulsar-perf simulation-server --port --service-url + * 4. Start the controller with pulsar-perf simulation-controller --cluster + * --servers : --server-port 5. You will get a shell on the + * controller, where you can use the commands trade, change, stop, trade_group, change_group, stop_group. You can enter + * "help" to see the syntax for the commands. Note that tenant, namespace, and topic refer to + * persistent://cluster/tenant/namespace/topic/bundle. For instance, to start trading for topic with destination + * persistent://mycluster/mytenant/mynamespace/mytopic/bundle at rate 200 msgs/s, you would type "trade mytenant + * mynamespace mytopic --rate 200". The group commands also refer to a "group_name" parameter. This is a string that is + * prefixed to the namespaces when trade_group is invoked so they may be identified by other group commands. At the + * moment, groups may not be modified after they have been created via trade_group. + * + */ +public class LoadSimulationController { + private static final Logger log = LoggerFactory.getLogger(LoadSimulationController.class); + private final static String QUOTA_ROOT = "/loadbalance/resource-quota/namespace"; + + // Input streams for each server to send commands through. + private final DataInputStream[] inputStreams; + + // Output streams for each server to receive information from. + private final DataOutputStream[] outputStreams; + + // Server host names. + private final String[] servers; + + // Port servers are listening on. + private final int serverPort; + + // The ZooKeeper cluster to run on. + private final String cluster; + + private final Random random; + + // JCommander arguments for starting a controller via main. + private static class MainArguments { + @Parameter(names = { "--cluster" }, description = "Cluster to test on", required = true) + String cluster; + + @Parameter(names = { "--servers" }, description = "Comma separated list of server hostnames", required = true) + String serverHostNames; + + @Parameter(names = { "--server-port" }, description = "Port that the servers are listening on", required = true) + int serverPort; + } + + // JCommander arguments for accepting user input. + private static class ShellArguments { + @Parameter(description = "Command arguments:\n" + "trade tenant namespace topic\n" + + "change tenant namespace topic\n" + "stop tenant namespace topic\n" + + "trade_group tenant group_name num_namespaces\n" + "change_group tenant group_name\n" + + "stop_group tenant group_name\n" + "script script_name\n" + "copy tenant_name source_zk target_zk\n" + + "stream source_zk\n", required = true) + List commandArguments; + + @Parameter(names = { "--rand-rate" }, description = "Choose message rate uniformly randomly from the next two " + + "comma separated values (overrides --rate)") + String rangeString = ""; + + @Parameter(names = { "--rate" }, description = "Messages per second") + double rate = 1; + + @Parameter(names = { "--rate-multiplier" }, description = "Multiplier to use for copying or streaming rates") + double rateMultiplier = 1; + + @Parameter(names = { "--size" }, description = "Message size in bytes") + int size = 1024; + + @Parameter(names = { "--separation" }, description = "Separation time in ms for trade_group actions " + + "(0 for no separation)") + int separation = 0; + + @Parameter(names = { "--topics-per-namespace" }, description = "Number of topics to create per namespace in " + + "trade_group (total number of topics is num_namespaces X num_topics)") + int topicsPerNamespace = 1; + } + + // In stream mode, the BrokerWatcher watches the /loadbalance/broker zpath + // and adds LoadReportWatchers + // accordingly when new brokers come up. + private class BrokerWatcher implements Watcher { + private final ZooKeeper zkClient; + private final Set brokers; + private final String path; + private final ShellArguments arguments; + + public BrokerWatcher(final String path, final ZooKeeper zkClient, final ShellArguments arguments) { + this.path = path; + this.zkClient = zkClient; + this.arguments = arguments; + brokers = new HashSet<>(); + process(null); + } + + public synchronized void process(final WatchedEvent event) { + try { + final List currentBrokers = zkClient.getChildren(path, this); + for (final String broker : currentBrokers) { + if (!brokers.contains(broker)) { + new LoadReportWatcher(String.format("%s/%s", path, broker), zkClient, arguments); + brokers.add(broker); + } + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + + // In stream mode, the LoadReportWatcher watches the /loadbalance/broker + // children and adds or modifies topics + // with suitable rates based on the most recent message rate and throughput + // information. + private class LoadReportWatcher implements Watcher { + private final ZooKeeper zkClient; + private final String path; + private final ShellArguments arguments; + + public LoadReportWatcher(final String path, final ZooKeeper zkClient, final ShellArguments arguments) { + this.path = path; + this.zkClient = zkClient; + this.arguments = arguments; + // Get initial topics and set this up as a watch by calling process. + process(null); + } + + public synchronized void process(final WatchedEvent event) { + try { + // Get the load report and put this back as a watch. + final LoadReport loadReport = ObjectMapperFactory.getThreadLocal() + .readValue(zkClient.getData(path, this, null), LoadReport.class); + for (final Map.Entry entry : loadReport.getBundleStats().entrySet()) { + final String bundle = entry.getKey(); + final String namespace = bundle.substring(0, bundle.lastIndexOf('/')); + final String destination = String.format("%s/%s", namespace, "t"); + final NamespaceBundleStats stats = entry.getValue(); + + // Approximate total message rate via average between + // in/out. + final double messageRate = arguments.rateMultiplier * (stats.msgRateIn + stats.msgRateOut) / 2; + + // size = throughput / rate. + final int messageSize = (int) Math.ceil(arguments.rateMultiplier + * (stats.msgThroughputIn + stats.msgThroughputOut) / (2 * messageRate)); + + final ShellArguments tradeArguments = new ShellArguments(); + arguments.rate = messageRate; + arguments.size = messageSize; + // Try to modify the topic if it already exists. Otherwise, + // create it. + if (!change(tradeArguments, destination)) { + trade(tradeArguments, destination); + } + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + + /** + * Create a LoadSimulationController with the given JCommander arguments. + * + * @param arguments + * Arguments to create from. + */ + public LoadSimulationController(final MainArguments arguments) throws Exception { + random = new Random(); + serverPort = arguments.serverPort; + cluster = arguments.cluster; + servers = arguments.serverHostNames.split(","); + final Socket[] sockets = new Socket[servers.length]; + inputStreams = new DataInputStream[servers.length]; + outputStreams = new DataOutputStream[servers.length]; + log.info("Found {} servers:", servers.length); + for (int i = 0; i < servers.length; ++i) { + sockets[i] = new Socket(servers[i], serverPort); + inputStreams[i] = new DataInputStream(sockets[i].getInputStream()); + outputStreams[i] = new DataOutputStream(sockets[i].getOutputStream()); + log.info("Connected to {}", servers[i]); + } + } + + // Check that the expected number of application arguments matches the + // actual number of application arguments. + private boolean checkAppArgs(final int numAppArgs, final int numRequired) { + if (numAppArgs != numRequired) { + log.info("ERROR: Wrong number of application arguments (found {}, required {})", numAppArgs, + numRequired); + return false; + } + return true; + } + + // Makes a destination string from a tenant name, namespace name, and topic + // name. + private String makeDestination(final String tenant, final String namespace, final String topic) { + return String.format("persistent://%s/%s/%s/%s", cluster, tenant, namespace, topic); + } + + // Write options that are common to modifying and creating topics. + private void writeProducerOptions(final DataOutputStream outputStream, final ShellArguments arguments, + final String destination) throws Exception { + if (!arguments.rangeString.isEmpty()) { + // If --rand-rate was specified, extract the bounds by splitting on + // the comma and parsing the resulting + // doubles. + final String[] splits = arguments.rangeString.split(","); + if (splits.length != 2) { + log.info("ERROR: Argument to --rand-rate should be a two comma-separated values"); + return; + } + final double first = Double.parseDouble(splits[0]); + final double second = Double.parseDouble(splits[1]); + final double min = Math.min(first, second); + final double max = Math.max(first, second); + arguments.rate = random.nextDouble() * (max - min) + min; + } + outputStream.writeUTF(destination); + outputStream.writeInt(arguments.size); + outputStream.writeDouble(arguments.rate); + } + + // Trade using the arguments parsed via JCommander and the destination name. + private synchronized void trade(final ShellArguments arguments, final String destination) throws Exception { + // Decide which server to send to randomly to preserve statelessness of + // the controller. + final int i = random.nextInt(servers.length); + log.info("Sending trade request to " + servers[i]); + outputStreams[i].write(LoadSimulationClient.TRADE_COMMAND); + writeProducerOptions(outputStreams[i], arguments, destination); + outputStreams[i].flush(); + if (inputStreams[i].read() != -1) { + log.info("Created producer and consumer for " + destination); + } else { + log.info("ERROR: Socket to {} closed", servers[i]); + } + } + + private void handleTrade(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Trade expects three application arguments: tenant, namespace, and + // topic. + if (checkAppArgs(commandArguments.size() - 1, 3)) { + final String destination = makeDestination(commandArguments.get(1), commandArguments.get(2), + commandArguments.get(3)); + trade(arguments, destination); + } + } + + // Change producer settings for a given destination and JCommander + // arguments. + // Returns true if the topic was found and false otherwise. + private synchronized boolean change(final ShellArguments arguments, final String destination) throws Exception { + log.info("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationClient.CHANGE_COMMAND); + writeProducerOptions(outputStream, arguments, destination); + outputStream.flush(); + } + boolean foundTopic = false; + for (int i = 0; i < servers.length; ++i) { + int readValue; + switch (readValue = inputStreams[i].read()) { + case LoadSimulationClient.FOUND_TOPIC: + log.info("Found topic {} on server {}", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationClient.NO_SUCH_TOPIC: + break; + case -1: + log.info("ERROR: Socket to {} closed", servers[i]); + break; + default: + log.info("ERROR: Unknown response signal received: " + readValue); + } + } + return foundTopic; + } + + private void handleChange(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Change expects three application arguments: tenant name, namespace + // name, and topic name. + if (checkAppArgs(commandArguments.size() - 1, 3)) { + final String destination = makeDestination(commandArguments.get(1), commandArguments.get(2), + commandArguments.get(3)); + if (!change(arguments, destination)) { + log.info("ERROR: Topic {} not found", destination); + } + } + } + + private void handleStop(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Stop expects three application arguments: tenant name, namespace + // name, and topic name. + if (checkAppArgs(commandArguments.size() - 1, 3)) { + final String destination = makeDestination(commandArguments.get(1), commandArguments.get(2), + commandArguments.get(3)); + log.info("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationClient.STOP_COMMAND); + outputStream.writeUTF(destination); + outputStream.flush(); + } + boolean foundTopic = false; + for (int i = 0; i < servers.length; ++i) { + int readValue; + switch (readValue = inputStreams[i].read()) { + case LoadSimulationClient.FOUND_TOPIC: + log.info("Found topic {} on server {}", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationClient.NO_SUCH_TOPIC: + break; + case LoadSimulationClient.REDUNDANT_COMMAND: + log.info("ERROR: Topic {} already stopped on {}", destination, servers[i]); + foundTopic = true; + break; + case -1: + log.info("ERROR: Socket to {} closed", servers[i]); + break; + default: + log.info("ERROR: Unknown response signal received: " + readValue); + } + } + if (!foundTopic) { + log.info("ERROR: Topic {} not found", destination); + } + } + } + + private void handleGroupTrade(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Group trade expects 3 application arguments: tenant name, group name, + // and number of namespaces. + if (checkAppArgs(commandArguments.size() - 1, 3)) { + final String tenant = commandArguments.get(1); + final String group = commandArguments.get(2); + final int numNamespaces = Integer.parseInt(commandArguments.get(3)); + for (int i = 0; i < numNamespaces; ++i) { + for (int j = 0; j < arguments.topicsPerNamespace; ++j) { + // For each namespace and topic pair, create the namespace + // by using the group name and the + // namespace index, and then create the topic by using the + // topic index. Then just call trade. + final String destination = makeDestination(tenant, String.format("%s-%d", group, i), + Integer.toString(j)); + trade(arguments, destination); + Thread.sleep(arguments.separation); + } + } + } + } + + private void handleGroupChange(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Group change expects two application arguments: tenant name and group + // name. + if (checkAppArgs(commandArguments.size() - 1, 2)) { + final String tenant = commandArguments.get(1); + final String group = commandArguments.get(2); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationClient.CHANGE_GROUP_COMMAND); + outputStream.writeUTF(tenant); + outputStream.writeUTF(group); + outputStream.writeInt(arguments.size); + outputStream.writeDouble(arguments.rate); + outputStream.flush(); + } + accumulateAndReport(tenant, group); + } + } + + // Report the number of topics found belonging to the given tenant and + // group. + private void accumulateAndReport(final String tenant, final String group) throws Exception { + int numFound = 0; + for (int i = 0; i < servers.length; ++i) { + final int foundOnServer = inputStreams[i].readInt(); + if (foundOnServer == -1) { + log.info("ERROR: Socket to {} closed", servers[i]); + } else if (foundOnServer == 0) { + log.info("Found no topics belonging to tenant {} and group {} on {}", tenant, group, + servers[i]); + } else if (foundOnServer > 0) { + log.info("Found {} topics belonging to tenant {} and group {} on {}", foundOnServer, tenant, + group, servers[i]); + numFound += foundOnServer; + } else { + log.info("ERROR: Negative value {} received for topic count on {}", foundOnServer, + servers[i]); + } + } + if (numFound == 0) { + log.info("ERROR: Found no topics belonging to tenant {} and group {}", tenant, group); + } else { + log.info("Found {} topics belonging to tenant {} and group {}", numFound, tenant, group); + } + } + + private void handleGroupStop(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Group stop requires two application arguments: tenant name and group + // name. + if (checkAppArgs(commandArguments.size() - 1, 2)) { + final String tenant = commandArguments.get(1); + final String group = commandArguments.get(2); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationClient.STOP_GROUP_COMMAND); + outputStream.writeUTF(tenant); + outputStream.writeUTF(group); + outputStream.flush(); + } + accumulateAndReport(tenant, group); + } + } + + // Recursively acquire all resource quotas by getting the ZK children of the + // given path and calling this function + // on the children if there are any, or getting the data from this ZNode + // otherwise. + private void getResourceQuotas(final String path, final ZooKeeper zkClient, + final Map bundleToQuota) throws Exception { + final List children = zkClient.getChildren(path, false); + if (children.isEmpty()) { + bundleToQuota.put(path, ObjectMapperFactory.getThreadLocal().readValue(zkClient.getData(path, false, null), + ResourceQuota.class)); + } else { + for (final String child : children) { + getResourceQuotas(String.format("%s/%s", path, child), zkClient, bundleToQuota); + } + } + } + + private void handleStream(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Stream accepts 1 application argument: ZooKeeper connect string. + if (checkAppArgs(commandArguments.size() - 1, 1)) { + final String zkConnectString = commandArguments.get(1); + final ZooKeeper zkClient = new ZooKeeper(zkConnectString, 5000, null); + new BrokerWatcher("/loadbalance/brokers", zkClient, arguments); + // This controller will now stream rate changes from the given ZK. + // Users wishing to stop this should Ctrl + C and use another + // Controller to send new commands. + while (true) + ; + } + } + + private void handleCopy(final ShellArguments arguments) throws Exception { + final List commandArguments = arguments.commandArguments; + // Copy accepts 3 application arguments: Tenant name, source ZooKeeper + // and target ZooKeeper connect strings. + if (checkAppArgs(commandArguments.size() - 1, 3)) { + final String tenantName = commandArguments.get(1); + final String sourceZKConnectString = commandArguments.get(2); + final String targetZKConnectString = commandArguments.get(3); + final ZooKeeper sourceZKClient = new ZooKeeper(sourceZKConnectString, 5000, null); + final ZooKeeper targetZKClient = new ZooKeeper(targetZKConnectString, 5000, null); + final Map bundleToQuota = new HashMap<>(); + getResourceQuotas(QUOTA_ROOT, sourceZKClient, bundleToQuota); + for (final Map.Entry entry : bundleToQuota.entrySet()) { + final String bundle = entry.getKey(); + final ResourceQuota quota = entry.getValue(); + // Simulation will send messages in and out at about the same + // rate, so just make the rate the average + // of in and out. + final double messageRate = (quota.getMsgRateIn() + quota.getMsgRateOut()) / 2; + final int messageSize = (int) Math + .ceil((quota.getBandwidthIn() + quota.getBandwidthOut()) / messageRate); + final int clusterStart = QUOTA_ROOT.length() + 1; + final int tenantStart = bundle.indexOf('/', clusterStart) + 1; + final String sourceCluster = bundle.substring(clusterStart, tenantStart - 1); + final int namespaceStart = bundle.indexOf('/', tenantStart) + 1; + final String sourceTenant = bundle.substring(tenantStart, namespaceStart - 1); + final String namespace = bundle.substring(namespaceStart, bundle.lastIndexOf('/')); + final String keyRangeString = bundle.substring(bundle.lastIndexOf('/') + 1); + // To prevent duplicate node issues for same namespace names in + // different clusters/tenants. + final String manglePrefix = String.format("%s-%s-%s", sourceCluster, sourceTenant, keyRangeString); + final String mangledNamespace = String.format("%s-%s", manglePrefix, namespace); + arguments.rate = messageRate * arguments.rateMultiplier; + arguments.size = messageSize; + final NamespaceBundleStats startingStats = new NamespaceBundleStats(); + + // Modify the original quota so that new rates are set. + quota.setMsgRateIn(quota.getMsgRateIn() * arguments.rateMultiplier); + quota.setMsgRateOut(quota.getMsgRateOut() * arguments.rateMultiplier); + quota.setBandwidthIn(quota.getBandwidthIn() * arguments.rateMultiplier); + quota.setBandwidthOut(quota.getBandwidthOut() * arguments.rateMultiplier); + + // Assume modified memory usage is comparable to the rate + // multiplier times the original usage. + quota.setMemory(quota.getMemory() * arguments.rateMultiplier); + startingStats.msgRateIn = quota.getMsgRateIn(); + startingStats.msgRateOut = quota.getMsgRateOut(); + startingStats.msgThroughputIn = quota.getBandwidthIn(); + startingStats.msgThroughputOut = quota.getBandwidthOut(); + final BundleData bundleData = new BundleData(10, 1000, startingStats); + // Assume there is ample history for topic. + bundleData.getLongTermData().setNumSamples(1000); + bundleData.getShortTermData().setNumSamples(1000); + final String oldAPITargetPath = String.format( + "/loadbalance/resource-quota/namespace/%s/%s/%s/0x00000000_0xffffffff", cluster, tenantName, + mangledNamespace); + final String newAPITargetPath = String.format("/loadbalance/bundle-data/%s/%s/%s/0x00000000_0xffffffff", + cluster, tenantName, mangledNamespace); + log.info("Copying {} to {}", bundle, oldAPITargetPath); + ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + log.info("Creating new API data at {}", newAPITargetPath); + // Put the quota in the new ZooKeeper. + ZkUtils.createFullPathOptimistic(targetZKClient, newAPITargetPath, bundleData.getJsonBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + trade(arguments, makeDestination(tenantName, mangledNamespace, "t")); + } + sourceZKClient.close(); + targetZKClient.close(); + } + } + + public void read(final String[] args) { + // Don't attempt to process blank input. + if (args.length > 0 && !(args.length == 1 && args[0].isEmpty())) { + final ShellArguments arguments = new ShellArguments(); + final JCommander jc = new JCommander(arguments); + try { + jc.parse(args); + final String command = arguments.commandArguments.get(0); + switch (command) { + case "trade": + handleTrade(arguments); + break; + case "change": + handleChange(arguments); + break; + case "stop": + handleStop(arguments); + break; + case "trade_group": + handleGroupTrade(arguments); + break; + case "change_group": + handleGroupChange(arguments); + break; + case "stop_group": + handleGroupStop(arguments); + break; + case "script": + // Read input from the given script instead of stdin until + // the script has executed completely. + final List commandArguments = arguments.commandArguments; + checkAppArgs(commandArguments.size() - 1, 1); + final String scriptName = commandArguments.get(1); + final BufferedReader scriptReader = new BufferedReader( + new InputStreamReader(new FileInputStream(Paths.get(scriptName).toFile()))); + String line = scriptReader.readLine(); + while (line != null) { + read(line.split("\\s+")); + line = scriptReader.readLine(); + } + scriptReader.close(); + break; + case "copy": + handleCopy(arguments); + break; + case "stream": + handleStream(arguments); + break; + case "quit": + case "exit": + System.exit(0); + break; + default: + log.info("ERROR: Unknown command \"{}\"", command); + } + } catch (ParameterException ex) { + ex.printStackTrace(); + jc.usage(); + } catch (Exception ex) { + ex.printStackTrace(); + } + } + } + + public void run() throws Exception { + BufferedReader inReader = new BufferedReader(new InputStreamReader(System.in)); + while (true) { + // Print the very simple prompt. + System.out.println(); + System.out.print("> "); + read(inReader.readLine().split("\\s+")); + } + } + + public static void main(String[] args) throws Exception { + final MainArguments arguments = new MainArguments(); + final JCommander jc = new JCommander(arguments); + try { + jc.parse(args); + } catch (Exception ex) { + jc.usage(); + ex.printStackTrace(); + System.exit(1); + } + (new LoadSimulationController(arguments)).run(); + } +} diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java new file mode 100644 index 0000000000000..afdb32d3d1712 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java @@ -0,0 +1,205 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.testclient; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.google.gson.Gson; +import com.yahoo.pulsar.broker.LocalBrokerData; +import com.yahoo.pulsar.broker.TimeAverageBrokerData; +import com.yahoo.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ModularLoadManagerBrokerMonitor { + private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); + private static final String BROKER_ROOT = "/loadbalance/brokers"; + private static final int ZOOKEEPER_TIMEOUT_MILLIS = 5000; + private final ZooKeeper zkClient; + private static final Gson gson = new Gson(); + + private static class BrokerWatcher implements Watcher { + public final ZooKeeper zkClient; + public Set brokers; + + public BrokerWatcher(final ZooKeeper zkClient) { + this.zkClient = zkClient; + this.brokers = Collections.emptySet(); + } + + public synchronized void process(final WatchedEvent event) { + try { + if (event.getType() == Event.EventType.NodeChildrenChanged) { + updateBrokers(event.getPath()); + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public synchronized void updateBrokers(final String path) { + final Set newBrokers = new HashSet<>(); + try { + newBrokers.addAll(zkClient.getChildren(path, this)); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + for (String oldBroker : brokers) { + if (!newBrokers.contains(oldBroker)) { + log.info("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + log.info("Gained broker: " + newBroker); + final BrokerDataWatcher brokerDataWatcher = new BrokerDataWatcher(zkClient); + brokerDataWatcher.printBrokerData(path + "/" + newBroker); + } + } + this.brokers = newBrokers; + } + } + + private static class BrokerDataWatcher implements Watcher { + private final ZooKeeper zkClient; + + public BrokerDataWatcher(final ZooKeeper zkClient) { + this.zkClient = zkClient; + } + + public static String brokerNameFromPath(final String path) { + return path.substring(path.lastIndexOf('/') + 1); + } + + public synchronized void process(final WatchedEvent event) { + try { + if (event.getType() == Event.EventType.NodeDataChanged) { + final String broker = brokerNameFromPath(event.getPath()); + printBrokerData(event.getPath()); + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + private static void printMessageData(final double msgThroughputIn, final double msgThroughputOut, + final double msgRateIn, final double msgRateOut) { + log.info(String.format("Message Throughput In: %.2f KB/s", msgThroughputIn / 1024)); + log.info(String.format("Message Throughput Out: %.2f KB/s", msgThroughputOut / 1024)); + log.info(String.format("Message Rate In: %.2f msgs/s", msgRateIn)); + log.info(String.format("Message Rate Out: %.2f msgs/s", msgRateOut)); + } + + public synchronized void printBrokerData(final String brokerPath) { + final String broker = brokerNameFromPath(brokerPath); + final String timeAveragePath = ModularLoadManagerImpl.TIME_AVERAGE_BROKER_ZPATH + "/" + broker; + LocalBrokerData localBrokerData; + try { + localBrokerData = gson.fromJson(new String(zkClient.getData(brokerPath, this, null)), + LocalBrokerData.class); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + + log.info("Broker Data for " + broker + ":"); + log.info("---------------"); + + log.info("Num Topics: " + localBrokerData.getNumTopics()); + log.info("Num Bundles: " + localBrokerData.getNumBundles()); + log.info("Num Consumers: " + localBrokerData.getNumConsumers()); + log.info("Num Producers: " + localBrokerData.getNumProducers()); + + log.info(String.format("CPU: %.2f%%", localBrokerData.getCpu().percentUsage())); + + log.info(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); + + log.info(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); + + log.info("Latest Data:"); + printMessageData(localBrokerData.getMsgThroughputIn(), localBrokerData.getMsgThroughputOut(), + localBrokerData.getMsgRateIn(), localBrokerData.getMsgRateOut()); + + TimeAverageBrokerData timeAverageData; + try { + timeAverageData = gson.fromJson(new String(zkClient.getData(timeAveragePath, null, null)), + TimeAverageBrokerData.class); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + log.info("Short Term Data:"); + printMessageData(timeAverageData.getShortTermMsgThroughputIn(), + timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), + timeAverageData.getShortTermMsgRateOut()); + + log.info("Long Term Data:"); + printMessageData(timeAverageData.getLongTermMsgThroughputIn(), + timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), + timeAverageData.getLongTermMsgRateOut()); + + if (!localBrokerData.getLastBundleGains().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleGains()) { + log.info("Gained Bundle: " + bundle); + } + } + if (!localBrokerData.getLastBundleLosses().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleLosses()) { + log.info("Lost Bundle: " + bundle); + } + } + } + } + + static class Arguments { + @Parameter(names = { "--connect-string" }, description = "Zookeeper connect string", required = true) + public String connectString = null; + } + + public ModularLoadManagerBrokerMonitor(final ZooKeeper zkClient) { + this.zkClient = zkClient; + } + + private void start() { + try { + final BrokerWatcher brokerWatcher = new BrokerWatcher(zkClient); + brokerWatcher.updateBrokers(BROKER_ROOT); + while (true) { + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public static void main(String[] args) { + try { + final Arguments arguments = new Arguments(); + final JCommander jc = new JCommander(arguments); + jc.parse(args); + final ZooKeeper zkClient = new ZooKeeper(arguments.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); + final ModularLoadManagerBrokerMonitor monitor = new ModularLoadManagerBrokerMonitor(zkClient); + monitor.start(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } +} diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/PerformanceConsumer.java index f9d0e40a93676..e412996e7f6d8 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/PerformanceConsumer.java @@ -132,11 +132,11 @@ public static void main(String[] args) throws Exception { if (arguments.serviceURL == null) { arguments.serviceURL = prop.getProperty("brokerServiceUrl"); } - + if (arguments.serviceURL == null) { arguments.serviceURL = prop.getProperty("webServiceUrl"); } - + // fallback to previous-version serviceUrl property to maintain backward-compatibility if (arguments.serviceURL == null) { arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/"); @@ -241,4 +241,4 @@ public void received(Consumer consumer, Message msg) { } private static final Logger log = LoggerFactory.getLogger(PerformanceConsumer.class); -} +} \ No newline at end of file diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java new file mode 100644 index 0000000000000..603d8b82d3009 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java @@ -0,0 +1,197 @@ +/** + * Copyright 2016 Yahoo Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.yahoo.pulsar.testclient; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.google.gson.Gson; +import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; +import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +/** + * To use the monitor, simply start one via pulsar-perf monitor --connect-string : You will then + * receive updates in LoadReports as they occur. + */ +public class SimpleLoadManagerBrokerMonitor { + private static final Logger log = LoggerFactory.getLogger(SimpleLoadManagerBrokerMonitor.class); + private static final String BROKER_ROOT = "/loadbalance/brokers"; + private static final int ZOOKEEPER_TIMEOUT_MILLIS = 5000; + private final ZooKeeper zkClient; + private static final Gson gson = new Gson(); + + private static class BrokerWatcher implements Watcher { + public final ZooKeeper zkClient; + public Set brokers; + + public BrokerWatcher(final ZooKeeper zkClient) { + this.zkClient = zkClient; + this.brokers = Collections.emptySet(); + } + + public synchronized void process(final WatchedEvent event) { + try { + if (event.getType() == Event.EventType.NodeChildrenChanged) { + updateBrokers(event.getPath()); + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public synchronized void updateBrokers(final String path) { + final Set newBrokers = new HashSet<>(); + try { + newBrokers.addAll(zkClient.getChildren(path, this)); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + for (String oldBroker : brokers) { + if (!newBrokers.contains(oldBroker)) { + log.info("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + log.info("Gained broker: " + newBroker); + final LoadReportWatcher loadReportWatcher = new LoadReportWatcher(zkClient); + loadReportWatcher.printLoadReport(path + "/" + newBroker); + } + } + this.brokers = newBrokers; + } + } + + private static class LoadReportWatcher implements Watcher { + private final ZooKeeper zkClient; + + public LoadReportWatcher(final ZooKeeper zkClient) { + this.zkClient = zkClient; + } + + public synchronized void process(final WatchedEvent event) { + try { + if (event.getType() == Event.EventType.NodeDataChanged) { + printLoadReport(event.getPath()); + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public synchronized void printLoadReport(final String path) { + final String brokerName = path.substring(path.lastIndexOf('/') + 1); + LoadReport loadReport; + try { + loadReport = gson.fromJson(new String(zkClient.getData(path, this, null)), LoadReport.class); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + final SystemResourceUsage resourceUsage = loadReport.getSystemResourceUsage(); + + log.info("Load Report for " + brokerName + ":"); + log.info("---------------"); + + log.info("Num Topics: " + loadReport.getNumTopics()); + log.info("Num Bundles: " + loadReport.getNumBundles()); + + log.info(String.format("Raw CPU: %.2f%%", resourceUsage.getCpu().percentUsage())); + log.info(String.format("Allocated CPU: %.2f%%", + percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); + log.info(String.format("Preallocated CPU: %.2f%%", + percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); + + log.info(String.format("Raw Memory: %.2f%%", resourceUsage.getMemory().percentUsage())); + log.info(String.format("Allocated Memory: %.2f%%", + percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); + log.info(String.format("Preallocated Memory: %.2f%%", + percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); + + log.info(String.format("Raw Bandwidth In: %.2f%%", resourceUsage.getBandwidthIn().percentUsage())); + log.info(String.format("Allocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + log.info(String.format("Preallocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + + log.info(String.format("Raw Bandwidth Out: %.2f%%", resourceUsage.getBandwidthOut().percentUsage())); + log.info(String.format("Allocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + log.info(String.format("Preallocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + + log.info(String.format("Direct Memory: %.2f%%", resourceUsage.getDirectMemory().percentUsage())); + + log.info(String.format("Messages In Per Second: %.2f", loadReport.getMsgRateIn())); + log.info(String.format("Messages Out Per Second: %.2f", loadReport.getMsgRateOut())); + log.info(String.format("Preallocated Messages In Per Second: %.2f", loadReport.getPreAllocatedMsgRateIn())); + log.info(String.format("Preallocated Out Per Second: %.2f", loadReport.getPreAllocatedMsgRateOut())); + + if (!loadReport.getBundleGains().isEmpty()) { + for (String bundle : loadReport.getBundleGains()) { + log.info("Gained Bundle: " + bundle); + } + } + if (!loadReport.getBundleLosses().isEmpty()) { + for (String bundle : loadReport.getBundleLosses()) { + log.info("Lost Bundle: " + bundle); + } + } + } + } + + static class Arguments { + @Parameter(names = { "--connect-string" }, description = "Zookeeper connect string", required = true) + public String connectString = null; + } + + public SimpleLoadManagerBrokerMonitor(final ZooKeeper zkClient) { + this.zkClient = zkClient; + } + + private static double percentUsage(final double usage, final double limit) { + return limit > 0 && usage >= 0 ? 100 * Math.min(1, usage / limit) : 0; + } + + private void start() { + try { + final BrokerWatcher brokerWatcher = new BrokerWatcher(zkClient); + brokerWatcher.updateBrokers(BROKER_ROOT); + while (true) { + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public static void main(String[] args) { + try { + final Arguments arguments = new Arguments(); + final JCommander jc = new JCommander(arguments); + jc.parse(args); + final ZooKeeper zkClient = new ZooKeeper(arguments.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); + final SimpleLoadManagerBrokerMonitor monitor = new SimpleLoadManagerBrokerMonitor(zkClient); + monitor.start(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } +}