From 1c46f5bb522ae95800c28b7779cddc1e63926ff4 Mon Sep 17 00:00:00 2001 From: breese Date: Tue, 21 Mar 2017 16:32:22 -0700 Subject: [PATCH 01/26] Introduce new load manager API --- bin/pulsar-perf | 8 + conf/broker.conf | 27 + .../pulsar/broker/ServiceConfiguration.java | 137 +++- .../com/yahoo/pulsar/broker/BrokerData.java | 51 ++ .../com/yahoo/pulsar/broker/BundleData.java | 66 ++ .../com/yahoo/pulsar/broker/JSONWritable.java | 31 + .../yahoo/pulsar/broker/LocalBrokerData.java | 289 +++++++++ .../yahoo/pulsar/broker/PulsarService.java | 47 +- .../pulsar/broker/TimeAverageBrokerData.java | 144 +++++ .../pulsar/broker/TimeAverageMessageData.java | 130 ++++ .../pulsar/broker/admin/BrokerStats.java | 4 +- .../yahoo/pulsar/broker/admin/Brokers.java | 2 +- .../broker/loadbalance/BrokerFilter.java | 30 + .../pulsar/broker/loadbalance/LoadData.java | 38 ++ .../broker/loadbalance/LoadManager.java | 20 + .../loadbalance/LoadReportUpdaterTask.java | 7 +- .../LoadResourceQuotaUpdaterTask.java | 7 +- .../loadbalance/LoadSheddingStrategy.java | 30 + .../broker/loadbalance/LoadSheddingTask.java | 7 +- .../broker/loadbalance/NewLoadManager.java | 66 ++ .../loadbalance/NewPlacementStrategy.java | 38 ++ .../loadbalance/impl/DeviationShedder.java | 119 ++++ .../impl/LeastLongTermMessageRate.java | 69 ++ .../impl/LinuxBrokerHostUsageImpl.java | 8 +- .../loadbalance/impl/NewLoadManagerImpl.java | 524 +++++++++++++++ .../impl/NewLoadManagerWrapper.java | 91 +++ .../impl/SimpleLoadManagerImpl.java | 352 ++++++---- .../broker/namespace/NamespaceService.java | 35 +- .../broker/namespace/ServiceUnitZkUtils.java | 2 +- .../pulsar/broker/service/BrokerService.java | 2 +- .../pulsar/broker/stats/MetricsGenerator.java | 2 +- .../broker/loadbalance/LoadBalancerTest.java | 22 +- .../SimpleLoadManagerImplTest.java | 6 +- .../client/api/BrokerServiceLookupTest.java | 30 +- .../impl/BrokerClientIntegrationTest.java | 2 +- .../data/loadbalancer/LoadReport.java | 154 ++++- .../loadbalancer/ResourceUnitRanking.java | 24 +- .../data/loadbalancer/ServiceLookupData.java | 12 + .../discovery/service/ServerConnection.java | 2 +- pulsar-testclient/pom.xml | 15 + .../pulsar/testclient/BrokerMonitor.java | 186 ++++++ .../testclient/LoadSimulationController.java | 610 ++++++++++++++++++ .../testclient/LoadSimulationServer.java | 330 ++++++++++ .../pulsar/testclient/NewBrokerMonitor.java | 193 ++++++ 44 files changed, 3744 insertions(+), 225 deletions(-) create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadData.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/DeviationShedder.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java create mode 100644 pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ServiceLookupData.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java diff --git a/bin/pulsar-perf b/bin/pulsar-perf index b64b046f10440..0425036bc257a 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -137,6 +137,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" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.BrokerMonitor "$@" +elif [ "$COMMAND" == "new-monitor" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.NewBrokerMonitor "$@" +elif [ "$COMMAND" == "simulation-server" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.LoadSimulationServer "$@" +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 53acb374ad25b..23aa0d0f0729a 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -293,3 +293,30 @@ keepAliveIntervalSeconds=30 # How often broker checks for inactive topics to be deleted (topics with no subscriptions and no one connected) brokerServicePurgeInactiveFrequencyInSeconds=60 + +# Number of samples to use for short term time window +numShortSamples=50 + +# Number of samples to use for long term time window +numLongSamples=1000 + +# How often in seconds to update the broker data +brokerDataUpdateIntervalSeconds=60 + +# How often in seconds to update the bundle data +bundleDataUpdateIntervalSeconds=60 + +# Default throughput in to assume for new bundles +defaultMsgThroughputIn=50000 + +# Default throughput out to assume for new bundles +defaultMsgThroughputOut=50000 + +# Default message rate in to assume for new bundles +defaultMsgRateIn=50 + +# Default message rate out to assume for new bundles +defaultMsgRateOut=50 + +# Name of load manager to use +loadManagerName=SimpleLoadManager 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 7c6541d217700..2d8f1ccfd5dd2 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 @@ -196,10 +196,14 @@ public class ServiceConfiguration implements PulsarConfiguration{ private boolean loadBalancerEnabled = false; // load placement strategy private String loadBalancerPlacementStrategy = "weightedRandomSelection"; // weighted random selection + // load placement secondary strategy (used to silently test an alternate strategy) + private String loadBalancerSecondaryStrategy = null; + // are all bundle placement operations forwarded to a lead broker + private boolean loadBalancerIsCentralized = false; // Percentage of change to trigger load report update private int loadBalancerReportUpdateThresholdPercentage = 10; // maximum interval to update load report - private int loadBalancerReportUpdateMaxIntervalMinutes = 15; + private int loadBalancerReportUpdateMaxIntervalMinutes = 1; // Frequency of report to collect private int loadBalancerHostUsageCheckIntervalMinutes = 1; // Load shedding interval. Broker periodically checks whether some traffic @@ -724,6 +728,22 @@ public String getLoadBalancerPlacementStrategy() { return this.loadBalancerPlacementStrategy; } + public void setLoadBalancerSecondaryStrategy(String secondaryStrategy) { + this.loadBalancerSecondaryStrategy = secondaryStrategy; + } + + public String getLoadBalancerSecondaryStrategy() { + return this.loadBalancerSecondaryStrategy; + } + + public void setLoadBalancerIsCentralized(boolean isCentralized) { + this.loadBalancerIsCentralized = isCentralized; + } + + public boolean getLoadBalancerIsCentralized() { + return this.loadBalancerIsCentralized; + } + public int getLoadBalancerReportUpdateThresholdPercentage() { return loadBalancerReportUpdateThresholdPercentage; } @@ -941,4 +961,117 @@ public String getReplicatorPrefix() { public void setReplicatorPrefix(String replicatorPrefix) { this.replicatorPrefix = replicatorPrefix; } + + + // Configurations for new load manager API + + // Number of samples to use for short term time window + private int numShortSamples = 50; + + // Number of samples to use for long term time window + private int numLongSamples = 1000; + + // How often in seconds to update the broker data + private long brokerDataUpdateIntervalSeconds = 60; + + // How often in seconds to update the bundle data + private long bundleDataUpdateIntervalSeconds = 60; + + // Default throughput in to assume for new bundles + private double defaultMsgThroughputIn = 50000; + + // Default throughput out to assume for new bundles + private double defaultMsgThroughputOut = 50000; + + // Default message rate in to assume for new bundles + private double defaultMsgRateIn = 50; + + // Default message rate out to assume for new bundles + private double defaultMsgRateOut = 50; + + // Name of load manager to use + private String loadManagerName = "SimpleLoadManager"; + + // Name of placement strategy to use for new loadbalancer API. + private String newPlacementStrategyName = "LeastLongTermMessageRate"; + + public int getNumShortSamples() { + return numShortSamples; + } + + public void setNumShortSamples(int numShortSamples) { + this.numShortSamples = numShortSamples; + } + + public int getNumLongSamples() { + return numLongSamples; + } + + public void setNumLongSamples(int numLongSamples) { + this.numLongSamples = numLongSamples; + } + + public long getBrokerDataUpdateIntervalSeconds() { + return brokerDataUpdateIntervalSeconds; + } + + public void setBrokerDataUpdateIntervalSeconds(long brokerDataUpdateIntervalSeconds) { + this.brokerDataUpdateIntervalSeconds = brokerDataUpdateIntervalSeconds; + } + + public long getBundleDataUpdateIntervalSeconds() { + return bundleDataUpdateIntervalSeconds; + } + + public void setBundleDataUpdateIntervalSeconds(long bundleDataUpdateIntervalSeconds) { + this.bundleDataUpdateIntervalSeconds = bundleDataUpdateIntervalSeconds; + } + + public double getDefaultMsgThroughputIn() { + return defaultMsgThroughputIn; + } + + public void setDefaultMsgThroughputIn(double defaultMsgThroughputIn) { + this.defaultMsgThroughputIn = defaultMsgThroughputIn; + } + + public double getDefaultMsgThroughputOut() { + return defaultMsgThroughputOut; + } + + public void setDefaultMsgThroughputOut(double defaultMsgThroughputOut) { + this.defaultMsgThroughputOut = defaultMsgThroughputOut; + } + + public double getDefaultMsgRateIn() { + return defaultMsgRateIn; + } + + public void setDefaultMsgRateIn(double defaultMsgRateIn) { + this.defaultMsgRateIn = defaultMsgRateIn; + } + + public double getDefaultMsgRateOut() { + return defaultMsgRateOut; + } + + public void setDefaultMsgRateOut(double defaultMsgRateOut) { + this.defaultMsgRateOut = defaultMsgRateOut; + } + + public String getLoadManagerName() { + return loadManagerName; + } + + public void setLoadManagerName(String loadManagerName) { + this.loadManagerName = loadManagerName; + } + + public String getNewPlacementStrategyName() { + return newPlacementStrategyName; + } + + public void setNewPlacementStrategyName(String newPlacementStrategyName) { + this.newPlacementStrategyName = newPlacementStrategyName; + } } 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..c5dc844e17f79 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java @@ -0,0 +1,51 @@ +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..81814fd6d86f3 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java @@ -0,0 +1,66 @@ +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..0079d66ed6a1a --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java @@ -0,0 +1,31 @@ +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..b742595f85bea --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -0,0 +1,289 @@ +package com.yahoo.pulsar.broker; + +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; + +import java.util.*; + +/** + * 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; + + // 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(); + 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; + } + + // 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 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 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..319bbb1bb2cce 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 @@ -24,13 +24,15 @@ 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.zookeeper.ZooKeeper; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.zookeeper.*; import org.eclipse.jetty.servlet.ServletHolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,6 +76,7 @@ */ public class PulsarService implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(PulsarService.class); + private static final String DYNAMIC_LOAD_MANAGER_ZPATH = "/loadbalance/settings/load-manager"; private ServiceConfiguration config = null; private NamespaceService nsservice = null; private ManagedLedgerClientFactory managedLedgerClientFactory = null; @@ -93,7 +96,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; @@ -201,6 +204,28 @@ public void close() throws PulsarServerException { } } + private class LoadManagerWatcher implements Watcher { + public void process(final WatchedEvent event) { + new Thread(() -> { + try { + LOG.info("Attempting to change load manager"); + final String newLoadManagerName = + new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, this, null)); + + config.setLoadManagerName(newLoadManagerName); + final LoadManager newLoadManager = LoadManager.create(config, PulsarService.this); + LOG.info("Created load manager: {}", newLoadManagerName); + loadManager.get().disableBroker(); + newLoadManager.start(); + loadManager.set(newLoadManager); + + } catch (Exception ex) { + LOG.warn("Failed to change load manager due to {}", ex); + } + }).start(); + } + } + /** * Get the current service configuration. * @@ -235,7 +260,10 @@ 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); + if (getZkClient().exists(DYNAMIC_LOAD_MANAGER_ZPATH, false) != null) { + config.setLoadManagerName(new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, false, null))); + } + this.loadManager = new AtomicReference<>(LoadManager.create(config, this)); this.startLoadManagementService(); @@ -309,6 +337,15 @@ public synchronized void brokerIsAFollowerNow() { acquireSLANamespace(); + try { + ZkUtils.createFullPathOptimistic(getZkClient(), DYNAMIC_LOAD_MANAGER_ZPATH, + config.getLoadManagerName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + // Ignore + } + + getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, new LoadManagerWatcher(), null); + LOG.info("messaging service is ready, bootstrap service on port={}, broker url={}, cluster={}, configs={}", config.getWebServicePort(), brokerServiceUrl, config.getClusterName(), config); } catch (Exception e) { @@ -396,7 +433,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 +662,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..b6f216b342ca8 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java @@ -0,0 +1,144 @@ +package com.yahoo.pulsar.broker; + +import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; + +import java.util.Map; +import java.util.Set; + +/** + * 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..a8f9052b875b1 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java @@ -0,0 +1,130 @@ +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..e0c99ace79dcd 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 @@ -152,7 +152,7 @@ public LoadReport getLoadReport() throws Exception { // Ensure super user access only validateSuperUserAccess(); try { - return ((SimpleLoadManagerImpl) pulsar().getLoadManager()).generateLoadReport(); + return ((SimpleLoadManagerImpl) 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,7 +169,7 @@ 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()); + SimpleLoadManagerImpl lm = (SimpleLoadManagerImpl) (pulsar().getLoadManager().get()); return lm.getResourceAvailabilityFor(ns).asMap(); } catch (Exception e) { log.error("Unable to get Resource Availability - [{}]", 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..cffca7fffb64b 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 @@ -68,7 +68,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(pulsar().getLoadManager().get().getBrokerRoot()); } 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..b2633d861351f --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java @@ -0,0 +1,30 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.ServiceConfiguration; + +import java.util.Set; + +/** + * 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); + + static BrokerFilter create(final ServiceConfiguration conf) { + // TODO + return null; + } +} 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..4aa3289606682 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadData.java @@ -0,0 +1,38 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.BrokerData; +import com.yahoo.pulsar.broker.BundleData; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * 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..578109e5b3ebf 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,6 +18,11 @@ 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.NewLoadManagerImpl; +import com.yahoo.pulsar.broker.loadbalance.impl.NewLoadManagerWrapper; +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; @@ -78,6 +83,11 @@ public interface LoadManager { */ void doNamespaceBundleSplit() throws Exception; + /** + * Determine the broker root. + */ + String getBrokerRoot(); + /** * Removes visibility of current broker from loadbalancer list so, other brokers can't redirect any request to this * broker and this broker won't accept new connection requests. @@ -87,4 +97,14 @@ public interface LoadManager { public void disableBroker() throws Exception; public void stop() throws PulsarServerException; + + static LoadManager create(final ServiceConfiguration conf, final PulsarService pulsar) { + switch (conf.getLoadManagerName()) { + case "NewLoadManager": + return new NewLoadManagerWrapper(new NewLoadManagerImpl(pulsar)); + case "SimpleLoadManager": + default: + 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..1d8238297f6d3 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java @@ -0,0 +1,30 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.ServiceConfiguration; + +import java.util.Map; +import java.util.Set; + +/** + * 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 selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); + + /** + * Create a LoadSheddingStrategy from the given configuration. + * @param conf The configuration to create the strategy from. + * @return The created strategy. + */ + static LoadSheddingStrategy create(final ServiceConfiguration conf) { + // TODO + return null; + } +} 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/NewLoadManager.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java new file mode 100644 index 0000000000000..018cdef58b807 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java @@ -0,0 +1,66 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.PulsarServerException; + +/** + * 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 NewLoadManager { + + /** + * 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(); + + /** + * Get the broker root ZooKeeper path. + */ + String getBrokerRoot(); + + /** + * As the leader broker, find a suitable broker for the assignment of the given bundle. + * @param bundleToAssign Full name of the bundle to assign. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + String selectBrokerForAssignment(String bundleToAssign); + + /** + * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. + */ + void updateLocalBrokerData(); + + /** + * 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, 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/NewPlacementStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java new file mode 100644 index 0000000000000..ea1f1135425f8 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java @@ -0,0 +1,38 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.BundleData; +import com.yahoo.pulsar.broker.ServiceConfiguration; +import com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; + +import java.util.Set; + +/** + * Interface which serves as a component for NewLoadManagerImpl, flexibly allowing the injection of potentially + * complex strategies. + */ +public interface NewPlacementStrategy { + + /** + * 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 NewPlacementStrategy create(final ServiceConfiguration conf) { + switch (conf.getNewPlacementStrategyName()) { + case "LeastLongTermMessageRate": + default: + return new LeastLongTermMessageRate(); + } + } +} 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..69d32e2160523 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/DeviationShedder.java @@ -0,0 +1,119 @@ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import com.yahoo.pulsar.broker.*; +import com.yahoo.pulsar.broker.loadbalance.LoadData; +import com.yahoo.pulsar.broker.loadbalance.LoadSheddingStrategy; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import java.util.*; + +/** + * An abstract class which makes a LoadSheddingStrategy which makes decisions based on standard deviation easier to + * implement. + */ +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 obsessive 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 selectBundlesForUnloading(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..2730d81703d84 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java @@ -0,0 +1,69 @@ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import com.yahoo.pulsar.broker.*; +import com.yahoo.pulsar.broker.loadbalance.LoadData; +import com.yahoo.pulsar.broker.loadbalance.NewPlacementStrategy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +/** + * Placement strategy which selects a broker based on which one has the least long term message rate. + */ +public class LeastLongTermMessageRate implements NewPlacementStrategy { + private static Logger log = LoggerFactory.getLogger(LeastLongTermMessageRate.class); + + // Maintain this list to reduce object creation. + private ArrayList bestBrokers; + + public LeastLongTermMessageRate() { + bestBrokers = new ArrayList<>(); + } + + // Form a score for a broker using its preallocated bundle data and time average data. + private static double getScore(final BrokerData brokerData) { + double totalMessageRate = 0; + for (BundleData bundleData: brokerData.getPreallocatedBundleData().values()) { + final TimeAverageMessageData longTermData = bundleData.getLongTermData(); + totalMessageRate += longTermData.getMsgRateIn() + longTermData.getMsgRateOut(); + } + final TimeAverageBrokerData timeAverageData = brokerData.getTimeAverageData(); + return totalMessageRate + timeAverageData.getLongTermMsgRateIn() + timeAverageData.getLongTermMsgRateOut(); + } + + /** + * 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)); + 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); + } + } + 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..d7571a1021caf 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 @@ -72,6 +72,7 @@ private void calculateBrokerHostUsage() { double totalNicUsageRx = getTotalNicUsageRxKb(nics); double totalCpuLimit = getTotalCpuLimit(); CpuStat cpuStat = getTotalCpuUsage(); + SystemResourceUsage usage = new SystemResourceUsage(); long now = System.currentTimeMillis(); @@ -159,8 +160,7 @@ private List getNics() { } private boolean isPhysicalNic(Path path) { - try { - path = Files.isSymbolicLink(path) ? Files.readSymbolicLink(path) : path; + path = Files.isSymbolicLink(path) ? path.toAbsolutePath() : path; if (!path.toString().contains("/virtual/")) { try { Files.readAllBytes(path.resolve("speed")); @@ -171,10 +171,6 @@ private boolean isPhysicalNic(Path path) { } } return false; - } catch (IOException e) { - LOG.error("Failed to read link target for NIC " + path, e); - return false; - } } private Path getNicSpeedPath(String nic) { diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java new file mode 100644 index 0000000000000..cf7a79af9d570 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java @@ -0,0 +1,524 @@ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import com.google.common.cache.*; +import com.yahoo.pulsar.broker.*; +import com.yahoo.pulsar.broker.loadbalance.*; +import com.yahoo.pulsar.client.admin.PulsarAdmin; +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; +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 java.io.IOException; +import java.net.URL; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkArgument; + +public class NewLoadManagerImpl implements NewLoadManager, ZooKeeperCacheListener { + public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; + public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; + public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; + + private static final int MIBI = 1024 * 1024; + private static final Logger log = LoggerFactory.getLogger(NewLoadManagerImpl.class); + + private final LocalBrokerData localData; + private final LoadData loadData; + + // Used to determine whether a bundle is preallocated. + private final Map preallocatedBundleToBroker; + + // Set of broker candidates to reuse so that object creation is avoided. + private final Set brokerCandidateCache; + + // Used to filter brokers from being selected for assignment. + private final List filterPipeline; + + // Pipeline used to determine what namespaces, if any, should be unloaded. + private final List loadSheddingPipeline; + + // Strategy used to determine where new topics should be placed. + private final NewPlacementStrategy placementStrategy; + + private final PulsarService pulsar; + private final ZooKeeper zkClient; + private final ServiceConfiguration conf; + private final BrokerHostUsage brokerHostUsage; + private final ZooKeeperDataCache brokerDataCache; + private final ZooKeeperChildrenCache availableActiveBrokers; + private final ScheduledExecutorService scheduler; + private final LoadingCache adminCache; + + // 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; + + // Timestamp of last invocation of updateBundleData. + private long lastBundleDataUpdate; + + private String brokerZnodePath; + private final String brokerRoot; + + // System resource usage directly after starting. + private SystemResourceUsage baselineSystemResourceUsage; + + public NewLoadManagerImpl(final PulsarService pulsar) { + this(pulsar, LOADBALANCE_BROKERS_ROOT); + } + + /** + * Initialize this load manager. + * @param pulsar Client to construct this manager from. + * @param brokerRoot ZooKeeper path containing some data implementing ServiceLookup. + */ + public NewLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { + this.brokerRoot = brokerRoot; + this.pulsar = pulsar; + zkClient = pulsar.getZkClient(); + conf = pulsar.getConfiguration(); + + loadData = new LoadData(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + brokerCandidateCache = new HashSet<>(); + filterPipeline = new ArrayList<>(); + loadSheddingPipeline = new ArrayList<>(); + localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + placementStrategy = NewPlacementStrategy.create(conf); + 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()); + } + }); + + + // Initialize the default + defaultStats = new NamespaceBundleStats(); + defaultStats.msgThroughputIn = conf.getDefaultMsgThroughputIn(); + defaultStats.msgThroughputOut = conf.getDefaultMsgThroughputOut(); + defaultStats.msgRateIn = conf.getDefaultMsgRateIn(); + defaultStats.msgRateOut = conf.getDefaultMsgRateOut(); + if (SystemUtils.IS_OS_LINUX) { + brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); + } else { + brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); + } + + 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); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); + 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(NewLoadManagerImpl.this::updateAll); + } + }); + scheduler = Executors.newScheduledThreadPool(1); + } + + /* + * 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", brokerRoot, broker); + final LocalBrokerData localData = brokerDataCache.get(key) + .orElseThrow(KeeperException.NoNodeException::new); + + if (brokerDataMap.containsKey(broker)) { + // Replace previous local broker data. + loadData.getBrokerData().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); + } + } + + /* + * Use the Pulsar client to acquire the namespace bundle stats. + */ + private Map getBundleStats() { + return pulsar.getBrokerService().getBundleStats(); + } + + /** + * Update both the broker data and the bundle data. + */ + public void updateAll() { + updateAllBrokerData(); + updateBundleData(); + } + + /** + * 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. + */ + public 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); + } + } + + // 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() + conf.getBrokerDataUpdateIntervalSeconds() * 1000; + } + + // Determine if the bundle data requires an update by measuring the time past since the last update. + private boolean needBundleDataUpdate() { + return System.currentTimeMillis() > lastBundleDataUpdate + conf.getBundleDataUpdateIntervalSeconds() * 1000; + } + + // 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. + } + } + } + + // Get the ZooKeeper path for the given bundle full name. + public static String getBundleDataZooKeeperPath(final String bundle) { + return BUNDLE_DATA_ZPATH + "/" + bundle; + } + + // Get the total number of used bytes in the JVM. + private static long getRealtimeJVMHeapUsageBytes() { + return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + } + + // Get the system resource usage for this broker. + 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(); + long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); + 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; + } + + // 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); + } + + // 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(conf.getNumShortSamples(), conf.getNumLongSamples(), defaultStats); + } + return bundleData; + } + + /** + * 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); + } + } + } + + 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()); + } + + /** + * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new + * brokers. + */ + @Override + public void doLoadShedding() { + for (LoadSheddingStrategy strategy: loadSheddingPipeline) { + final Map bundlesToUnload = strategy.selectBundlesForUnloading(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( + getNamespaceNameFromBundleName(bundle), 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? + } + + @Override + public String getBrokerRoot() { + return brokerRoot; + } + + /** + * 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 bundle Full name of the bundle to assign. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + @Override + public synchronized String selectBrokerForAssignment(final String bundle) { + // ?: Is it too inefficient to make this synchronized? If so, it may be a good idea to use weighted random + // or atomic data. + 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(); + brokerCandidateCache.addAll(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, brokerRoot); + + String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); + brokerZnodePath = brokerRoot + "/" + 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(); + baselineSystemResourceUsage = getSystemResourceUsage(); + } 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 = getSystemResourceUsage(); + 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/NewLoadManagerWrapper.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java new file mode 100644 index 0000000000000..f777eabf9b242 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java @@ -0,0 +1,91 @@ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; +import com.yahoo.pulsar.broker.loadbalance.NewLoadManager; +import com.yahoo.pulsar.broker.loadbalance.LoadManager; +import com.yahoo.pulsar.broker.loadbalance.ResourceUnit; +import com.yahoo.pulsar.broker.stats.Metrics; +import com.yahoo.pulsar.common.naming.ServiceUnitId; + +import java.util.Collections; +import java.util.List; + +/** + * Wrapper class allowing classes of instance NewLoadManager to be compatible with the interface LoadManager. + */ +public class NewLoadManagerWrapper implements LoadManager { + private NewLoadManager loadManager; + + public NewLoadManagerWrapper(final NewLoadManager 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 String getBrokerRoot() { + return loadManager.getBrokerRoot(); + } + + @Override + public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { + return new SimpleResourceUnit(String.format("http://%s", + loadManager.selectBrokerForAssignment(serviceUnit.toString())), + new PulsarResourceDescription()); + } + + @Override + public List getLoadBalancingMetrics() { + return Collections.emptyList(); + } + + @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..520d6fe84d414 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 @@ -120,8 +120,8 @@ 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; @@ -136,6 +136,8 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/brokers"; public static final String LOADBALANCER_DYNAMIC_SETTING_STRATEGY_ZPATH = "/loadbalance/settings/strategy"; + public static final String LOADBALANCER_DYNAMIC_SETTING_SECONDARY_STRATEGY_ZPATH = "/loadbalance/settings/secondary_strategy"; + public static final String LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH = "/loadbalance/settings/is_centralized"; 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"; private static final String LOADBALANCER_DYNAMIC_SETTING_OVERLOAD_THRESHOLD_ZPATH = "/loadbalance/settings/overload_threshold"; @@ -145,6 +147,8 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private static final String SETTING_NAME_LOAD_FACTOR_CPU = "loadFactorCPU"; private static final String SETTING_NAME_LOAD_FACTOR_MEM = "loadFactorMemory"; private static final String SETTING_NAME_STRATEGY = "loadBalancerStrategy"; + private static final String SETTING_NAME_SECONDARY_STRATEGY = "loadBalancerSecondaryStrategy"; + private static final String SETTING_NAME_IS_CENTRALIZED = "loadBalancerIsCentralized"; private static final String SETTING_NAME_OVERLOAD_THRESHOLD = "overloadThreshold"; private static final String SETTING_NAME_UNDERLOAD_THRESHOLD = "underloadThreshold"; private static final String SETTING_NAME_COMFORTLOAD_THRESHOLD = "comfortLoadThreshold"; @@ -152,8 +156,10 @@ 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 String brokerRoot; private final ScheduledExecutorService scheduler; private final ZooKeeperChildrenCache availableActiveBrokers; @@ -168,6 +174,11 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private boolean forceLoadReportUpdate = false; public SimpleLoadManagerImpl(PulsarService pulsar) { + this(pulsar, LOADBALANCE_BROKERS_ROOT); + } + + public SimpleLoadManagerImpl(PulsarService pulsar, final String brokerRoot) { + this.brokerRoot = brokerRoot; this.policies = new SimpleResourceAllocationPolicies(pulsar); this.sortedRankings.set(new TreeMap<>()); this.currentLoadReports = new HashMap<>(); @@ -177,7 +188,7 @@ public SimpleLoadManagerImpl(PulsarService pulsar) { this.realtimeAvgResourceQuota = new ResourceQuota(); placementStrategy = new WRRPlacementStrategy(); lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); if (SystemUtils.IS_OS_LINUX) { brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); } else { @@ -205,7 +216,7 @@ 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(); @@ -216,7 +227,7 @@ public Long load(String key) throws Exception { return System.currentTimeMillis(); } }); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LOADBALANCE_BROKERS_ROOT); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override public void onUpdate(String path, Set data, Stat stat) { @@ -230,23 +241,29 @@ public void onUpdate(String path, Set data, Stat stat) { this.pulsar = pulsar; } + @Override + public String getBrokerRoot() { + return brokerRoot; + } + @Override public void start() throws PulsarServerException { try { // Register the brokers in zk list ServiceConfiguration conf = pulsar.getConfiguration(); - if (pulsar.getZkClient().exists(LOADBALANCE_BROKERS_ROOT, false) == null) { + if (pulsar.getZkClient().exists(brokerRoot, false) == null) { try { - ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), LOADBALANCE_BROKERS_ROOT, new byte[0], - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerRoot, new byte[0], + Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } catch (KeeperException.NodeExistsException e) { // ignore the exception, node might be present already } } - + Thread.sleep(5000); String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); - brokerZnodePath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; + brokerZnodePath = brokerRoot + "/" + lookupServiceAddress; LoadReport loadReport = null; + Thread.sleep(5000); try { loadReport = generateLoadReport(); this.lastResourceUsageTimestamp = loadReport.getTimestamp(); @@ -259,7 +276,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 +290,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 +323,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 +368,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,44 +378,51 @@ 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 this.getDynamicConfigurationBoolean(LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH, + SETTING_NAME_IS_CENTRALIZED, + pulsar.getConfiguration().getLoadBalancerIsCentralized()); } 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()); } + /* * temp method, remove it in future, in-place to make this glue code to make load balancing work */ @@ -460,15 +485,15 @@ private ResourceQuota timeSmoothQuota(ResourceQuota oldQuota, double msgRateIn, 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 +530,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 +547,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 +576,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 +592,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); @@ -661,16 +684,18 @@ private synchronized void doLoadRanking() { ResourceQuota allocatedQuota = getTotalAllocatedQuota(loadedBundles); ResourceQuota preAllocatedQuota = getTotalAllocatedQuota(preAllocatedBundles); ResourceUnitRanking ranking = new ResourceUnitRanking(loadReport.getSystemResourceUsage(), - loadedBundles, allocatedQuota, preAllocatedBundles, preAllocatedQuota); + loadedBundles, allocatedQuota, preAllocatedBundles, preAllocatedQuota); newResourceUnitRankings.put(resourceUnit, ranking); // generated sorted ranking double loadPercentage = ranking.getEstimatedLoadPercentage(); long maxCapacity = ranking.estimateMaxCapacity( - pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); + 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); @@ -694,7 +719,7 @@ private synchronized void doLoadRanking() { } } else { log.info("Leader broker[{}] No ResourceUnits to rank this run, Using Old Ranking", - pulsar.getWebServiceAddress()); + pulsar.getWebServiceAddress()); } } @@ -739,7 +764,7 @@ 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,12 +774,18 @@ private ResourceUnit findBrokerForPlacement(Multimap candida ResourceUnit idleRU = null; ResourceUnit maxAvailableRU = null; ResourceUnit randomRU = null; - ResourceUnit selectedRU = null; + ResourceUnitRanking selectedRanking = null; String serviceUnitId = serviceUnit.toString(); + // 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); 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(); @@ -802,7 +833,7 @@ private ResourceUnit findBrokerForPlacement(Multimap candida selectedRanking = ranking; minLoadPercentage = loadPercentage; } else { - if (ranking.compareTo(selectedRanking) < 0) { + if ((unboundedRanks ? ranking.compareMessageRateTo(selectedRanking) : ranking.compareTo(selectedRanking)) < 0) { minLoadPercentage = loadPercentage; selectedRU = candidate; selectedRanking = ranking; @@ -815,10 +846,10 @@ private ResourceUnit findBrokerForPlacement(Multimap candida // 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) { + } else if (minLoadPercentage >= 100.0 && randomRU != null && !unboundedRanks) { // all brokers are full, assign to a random one selectedRU = randomRU; - } else if (minLoadPercentage > overloadThreshold) { + } else if (minLoadPercentage > overloadThreshold && !unboundedRanks) { // assign to the broker with maximum available capacity if all brokers are overloaded selectedRU = maxAvailableRU; } @@ -832,6 +863,7 @@ private ResourceUnit findBrokerForPlacement(Multimap candida if (!ranking.isServiceUnitPreAllocated(serviceUnitId)) { ResourceQuota quota = this.getResourceQuota(serviceUnitId); ranking.addPreAllocatedServiceUnit(serviceUnitId, quota); + resourceUnitRankings.put(selectedRU, ranking); } } } @@ -845,9 +877,9 @@ private Multimap getFinalCandidatesWithPolicy(NamespaceName 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()); + "Not enough of primaries [{}] available for namespace - [{}], " + + "adding shared [{}] as possible candidate owners", + primaries.size(), namespace.toString(), shared.size()); finalCandidates.putAll(shared); } return finalCandidates; @@ -874,7 +906,7 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni 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); + entry.getKey(), serviceUnit); URL brokerUrl = null; try { brokerUrl = new URL(String.format(ru.getResourceId())); @@ -889,22 +921,22 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni matchedPrimaries.put(entry.getKey(), ru); if (log.isDebugEnabled()) { log.debug( - "Added Primary Broker - [{}] as possible Candidates for" - + " namespace - [{}] with policies", - brokerUrl.getHost(), namespace.toString()); + "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()); + "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()); + + " for namespace - [{}] ", brokerUrl.getHost(), namespace.toString()); } } @@ -912,7 +944,7 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni if (policies.isSharedBroker(brokerUrl.getHost())) { matchedShared.put(entry.getKey(), ru); log.debug("Added Shared Broker - [{}] as possible Candidates for namespace - [{}]", - brokerUrl.getHost(), namespace.toString()); + brokerUrl.getHost(), namespace.toString()); } } } @@ -921,9 +953,9 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni 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()); + "Policies not present for namespace - [{}] so only " + + "considering shared [{}] brokers for possible owner", + namespace.toString(), matchedShared.size()); return getFinalCandidatesNoPolicy(matchedShared); } } @@ -943,7 +975,7 @@ private Map> getAvailableBrokers(ServiceUnitId serviceUn if (availableBrokers.isEmpty()) { // Create a map with all available brokers with no load information - Set activeBrokers = availableActiveBrokers.get(LOADBALANCE_BROKERS_ROOT); + Set activeBrokers = availableActiveBrokers.get(brokerRoot); List brokersToShuffle = new ArrayList<>(activeBrokers); Collections.shuffle(brokersToShuffle); activeBrokers = new HashSet<>(brokersToShuffle); @@ -951,7 +983,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()); @@ -981,13 +1013,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 @@ -1015,11 +1048,11 @@ private void updateRanking() { Set activeBrokers = availableActiveBrokers.get(); for (String broker : activeBrokers) { try { - String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker); + String key = String.format("%s/%s", brokerRoot, 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 +1068,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() { @@ -1075,7 +1108,7 @@ private SystemResourceUsage getSystemResourceUsage() throws IOException { // Collect JVM direct memory systemResourceUsage.directMemory.usage = (double) (sun.misc.SharedSecrets.getJavaNioAccess() - .getDirectBufferPool().getMemoryUsed() / MBytes); + .getDirectBufferPool().getMemoryUsed() / MBytes); systemResourceUsage.directMemory.limit = (double) (sun.misc.VM.maxDirectMemory() / MBytes); return systemResourceUsage; @@ -1087,20 +1120,67 @@ public LoadReport generateLoadReport() throws Exception { if (timeSinceLastGenMillis <= LOAD_REPORT_UPDATE_MIMIMUM_INTERVAL) { return lastLoadReport; } - try { LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + 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())); + isAboveLoadLevel(systemResourceUsage, this.getLoadBalancerBrokerOverloadedThresholdPercentage())); loadReport.setUnderLoaded( - isBelowLoadLevel(systemResourceUsage, this.getLoadBalancerBrokerUnderloadedThresholdPercentage())); + 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(); + final Set bundleGains = new HashSet<>(); + final Set bundleLosses = new HashSet<>(); + + for (String oldBundle: oldBundles) { + if (!newBundles.contains(oldBundle)) { + bundleLosses.add(oldBundle); + } + } + + for (String newBundle: newBundles) { + if (!oldBundles.contains(newBundle)) { + bundleGains.add(newBundle); + } + } + loadReport.setBundleGains(bundleGains); + loadReport.setBundleLosses(bundleLosses); + + 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); @@ -1145,49 +1225,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 +1275,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 @@ -1220,7 +1299,6 @@ private String getBundleRangeFromBundleName(String bundleName) { // 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)); Map> availableBrokers = sortedRankings.get(); // this does not have "http://" in front, hacky but no time to pretty up @@ -1250,10 +1328,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 +1345,17 @@ 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)); + getNamespaceNameFromBundleName(bundleName), 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 +1368,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 +1383,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 +1394,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 +1423,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,7 +1441,7 @@ 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 { @@ -1371,7 +1449,7 @@ public void doNamespaceBundleSplit() throws Exception { 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 +1459,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 +1474,7 @@ public void doNamespaceBundleSplit() throws Exception { for (String bundleName : bundlesToBeSplit) { try { pulsar.getAdminClient().namespaces().splitNamespaceBundle( - getNamespaceNameFromBundleName(bundleName), getBundleRangeFromBundleName(bundleName)); + getNamespaceNameFromBundleName(bundleName), 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..f65f544c1ddd3 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.get().getBrokerRoot(), 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.get().getBrokerRoot()); 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 4e813c8ef9344..d47cedfafa030 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 @@ -334,7 +334,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 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..4a059135b9d72 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 @@ -115,12 +115,12 @@ 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); @@ -202,8 +202,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 +214,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); @@ -286,12 +286,12 @@ 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); @@ -372,11 +372,11 @@ 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); 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 6c6dba930c904..a9cda3f7cc4f6 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 @@ -158,7 +158,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..31e6b7d029e53 --- /dev/null +++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/policies/data/loadbalancer/ServiceLookupData.java @@ -0,0 +1,12 @@ +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..4bb16cc6eadc8 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -38,11 +38,26 @@ test-jar test + + org.apache.zookeeper + zookeeper + 3.4.8 + + + ${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/BrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java new file mode 100644 index 0000000000000..69596a7a61b37 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java @@ -0,0 +1,186 @@ +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 java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ServerSocket; +import java.net.Socket; +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 BrokerMonitor { + 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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker: newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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(); + + System.out.println("\nLoad Report for " + brokerName + ":"); + System.out.println("---------------"); + + + System.out.println("\nNum Topics: " + loadReport.getNumTopics()); + System.out.println("Num Bundles: " + loadReport.getNumBundles()); + + System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); + System.out.println(String.format("Allocated CPU: %.2f%%", + percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); + System.out.println(String.format("Preallocated CPU: %.2f%%", + percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); + + System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); + System.out.println(String.format("Allocated Memory: %.2f%%", + percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); + System.out.println(String.format("Preallocated Memory: %.2f%%", + percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); + + System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); + System.out.println(String.format("Allocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + + System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); + System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + + System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); + + System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); + System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); + System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); + System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); + System.out.println(); + if (!loadReport.getBundleGains().isEmpty()) { + for (String bundle: loadReport.getBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!loadReport.getBundleLosses().isEmpty()) { + for (String bundle: loadReport.getBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + static class Arguments { + @Parameter(names = {"--connect-string"}, description = "Zookeeper connect string", required = true) + public String connectString = null; + } + + public BrokerMonitor(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 BrokerMonitor monitor = new BrokerMonitor(zkClient); + monitor.start(); + } 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..5fd1c4926d5c6 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -0,0 +1,610 @@ +package com.yahoo.pulsar.testclient; + +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.apache.bookkeeper.util.ZkUtils; +import org.apache.zookeeper.*; + +import java.io.*; +import java.net.Socket; +import java.nio.file.Paths; +import java.util.*; + +/** + * 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 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]; + System.out.format("Found %d servers:\n", 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()); + System.out.format("Connected to %s\n", 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) { + System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", + 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) { + System.out.println("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); + System.out.println("Sending trade request to " + servers[i]); + outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); + writeProducerOptions(outputStreams[i], arguments, destination); + outputStreams[i].flush(); + if (inputStreams[i].read() != -1) { + System.out.println("Created producer and consumer for " + destination); + } else { + System.out.format("ERROR: Socket to %s closed\n", 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 { + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("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)) { + System.out.format("ERROR: Topic %s not found\n", 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)); + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case LoadSimulationServer.REDUNDANT_COMMAND: + System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); + foundTopic = true; + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("ERROR: Unknown response signal received: " + readValue); + } + } + if (!foundTopic) { + System.out.format("ERROR: Topic %s not found\n", 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(LoadSimulationServer.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) { + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + } else if (foundOnServer == 0) { + System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, + servers[i]); + } else if (foundOnServer > 0){ + System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, + tenant, group, servers[i]); + numFound += foundOnServer; + } else { + System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, + servers[i]); + } + } + if (numFound == 0) { + System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); + } else { + System.out.format("Found %d topics belonging to tenant %s and group %s\n", 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(LoadSimulationServer.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); + System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); + ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + System.out.format("Creating new API data at %s\n", 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: + System.out.format("ERROR: Unknown command \"%s\"\n", 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/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java new file mode 100644 index 0000000000000..79dfb956d1aee --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -0,0 +1,330 @@ +package com.yahoo.pulsar.testclient; + +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.*; +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.apache.commons.lang.SystemUtils; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +/** + * LoadSimulationServer 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 LoadSimulationServer { + // 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.createProducerAsync(topic, producerConf).get(); + } 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 LoadSimulationServer. + 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 LoadSimulationServer(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(); + + // Disable connection pooling. + clientConf.setConnectionsPerBroker(0); + + // 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 LoadSimulationServer(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. + System.out.println("Listening for controller command..."); + final Socket socket = serverSocket.accept(); + System.out.format("Connected to %s\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/NewBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java new file mode 100644 index 0000000000000..9fcc8048637b6 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java @@ -0,0 +1,193 @@ +package com.yahoo.pulsar.testclient; + +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.NewLoadManagerImpl; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +public class NewBrokerMonitor { + private static final String BROKER_ROOT = "/loadbalance/new-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.EMPTY_SET; + } + + 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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker: newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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) { + System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); + System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); + System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); + System.out.format("Message Rate Out: %.2f msgs/s\n", msgRateOut); + } + + public synchronized void printBrokerData(final String brokerPath) { + final String broker = brokerNameFromPath(brokerPath); + final String timeAveragePath = NewLoadManagerImpl.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); + } + + System.out.println("\nBroker Data for " + broker + ":"); + System.out.println("---------------"); + + + System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); + System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); + System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); + System.out.println("Num Producers: " + localBrokerData.getNumProducers()); + + System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); + + System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); + + System.out.println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); + + System.out.println("\nLatest Data:\n"); + 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); + } + System.out.println("\nShort Term Data:\n"); + printMessageData(timeAverageData.getShortTermMsgThroughputIn(), + timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), + timeAverageData.getShortTermMsgRateOut()); + + System.out.println("\nLong Term Data:\n"); + printMessageData(timeAverageData.getLongTermMsgThroughputIn(), + timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), + timeAverageData.getLongTermMsgRateOut()); + + + System.out.println(); + if (!localBrokerData.getLastBundleGains().isEmpty()) { + for (String bundle: localBrokerData.getLastBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!localBrokerData.getLastBundleLosses().isEmpty()) { + for (String bundle: localBrokerData.getLastBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + static class Arguments { + @Parameter(names = {"--connect-string"}, description = "Zookeeper connect string", required = true) + public String connectString = null; + } + + public NewBrokerMonitor(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 NewBrokerMonitor monitor = new NewBrokerMonitor(zkClient); + monitor.start(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } +} From bbdf123d75396b57715911f22c521d6602a946e2 Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 22 Mar 2017 14:02:38 -0700 Subject: [PATCH 02/26] Fix formatting and style issues --- bin/pulsar-perf | 17 +- conf/broker.conf | 21 +- .../pulsar/broker/ServiceConfiguration.java | 103 +- .../com/yahoo/pulsar/broker/BrokerData.java | 87 +- .../com/yahoo/pulsar/broker/BundleData.java | 111 +- .../com/yahoo/pulsar/broker/JSONWritable.java | 38 +- .../yahoo/pulsar/broker/LocalBrokerData.java | 565 ++++---- .../yahoo/pulsar/broker/PulsarService.java | 11 +- .../pulsar/broker/TimeAverageBrokerData.java | 282 ++-- .../pulsar/broker/TimeAverageMessageData.java | 261 ++-- .../pulsar/broker/admin/BrokerStats.java | 12 +- .../broker/loadbalance/BrokerFilter.java | 43 +- .../pulsar/broker/loadbalance/LoadData.java | 51 +- .../broker/loadbalance/LoadManager.java | 166 +-- .../loadbalance/LoadSheddingStrategy.java | 36 +- .../broker/loadbalance/NewLoadManager.java | 66 - .../loadbalance/NewPlacementStrategy.java | 38 - .../loadbalance/impl/DeviationShedder.java | 218 +-- .../impl/LeastLongTermMessageRate.java | 123 +- .../loadbalance/impl/NewLoadManagerImpl.java | 524 ------- .../impl/NewLoadManagerWrapper.java | 91 -- .../impl/SimpleLoadManagerImpl.java | 2 - .../data/loadbalancer/ServiceLookupData.java | 8 +- .../pulsar/testclient/BrokerMonitor.java | 186 --- .../testclient/LoadSimulationController.java | 1227 +++++++++-------- .../testclient/LoadSimulationServer.java | 655 ++++----- .../pulsar/testclient/NewBrokerMonitor.java | 193 --- .../testclient/PerformanceConsumer.java | 6 +- 28 files changed, 2110 insertions(+), 3031 deletions(-) delete mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java delete mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java delete mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java delete mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java delete mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java delete mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java diff --git a/bin/pulsar-perf b/bin/pulsar-perf index 0425036bc257a..dd39891d67e0b 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -73,8 +73,13 @@ pulsar_help() { cat < where command is one of: - produce Run a producer - consume Run a consumer + produce Run a producer + consume Run a consumer + simple-monitor Continuously receive broker data when using SimpleLoadManagerImpl + modular-monitor Continuously receive broker data when using ModularLoadManagerImpl + simulation-server 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,10 +142,10 @@ 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" ]; then - exec $JAVA $OPTS com.yahoo.pulsar.testclient.BrokerMonitor "$@" -elif [ "$COMMAND" == "new-monitor" ]; then - exec $JAVA $OPTS com.yahoo.pulsar.testclient.NewBrokerMonitor "$@" +elif [ "$COMMAND" == "simple-monitor" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.SimpleLoadManagerBrokerMonitor "$@" +elif [ "$COMMAND" == "modular-monitor" ]; then + exec $JAVA $OPTS com.yahoo.pulsar.testclient.ModularLoadManagerBrokerMonitor "$@" elif [ "$COMMAND" == "simulation-server" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.LoadSimulationServer "$@" elif [ "$COMMAND" == "simulation-controller" ]; then diff --git a/conf/broker.conf b/conf/broker.conf index 23aa0d0f0729a..1e757d6e71f90 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -295,28 +295,31 @@ keepAliveIntervalSeconds=30 brokerServicePurgeInactiveFrequencyInSeconds=60 # Number of samples to use for short term time window -numShortSamples=50 +loadManagerNumberOfSamplesShortTermWindow=50 # Number of samples to use for long term time window -numLongSamples=1000 +loadManagerNumberOfSamplesLongTermWindow=1000 # How often in seconds to update the broker data -brokerDataUpdateIntervalSeconds=60 +loadManagerBrokerDataUpdateIntervalInSeconds=60 # How often in seconds to update the bundle data -bundleDataUpdateIntervalSeconds=60 +loadManagerBundleDataUpdateIntervalInSeconds=60 # Default throughput in to assume for new bundles -defaultMsgThroughputIn=50000 +loadManagerDefaultMessageThroughputIn=50000 # Default throughput out to assume for new bundles -defaultMsgThroughputOut=50000 +loadManagerDefaultMessageThroughputOut=50000 # Default message rate in to assume for new bundles -defaultMsgRateIn=50 +loadManagerDefaultMessageRateIn=50 # Default message rate out to assume for new bundles -defaultMsgRateOut=50 +loadManagerDefaultMessageRateOut=50 # Name of load manager to use -loadManagerName=SimpleLoadManager +loadManagerClassName=com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl + +# Name of placement strategy to use for new load manager API +loadManagerPlacementStrategyClassName=com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate 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 2d8f1ccfd5dd2..c6700a42363f2 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 @@ -203,7 +203,7 @@ public class ServiceConfiguration implements PulsarConfiguration { // Percentage of change to trigger load report update private int loadBalancerReportUpdateThresholdPercentage = 10; // maximum interval to update load report - private int loadBalancerReportUpdateMaxIntervalMinutes = 1; + private int loadBalancerReportUpdateMaxIntervalMinutes = 15; // Frequency of report to collect private int loadBalancerHostUsageCheckIntervalMinutes = 1; // Load shedding interval. Broker periodically checks whether some traffic @@ -966,112 +966,113 @@ public void setReplicatorPrefix(String replicatorPrefix) { // Configurations for new load manager API // Number of samples to use for short term time window - private int numShortSamples = 50; + private int loadManagerNumberOfSamplesShortTermWindow = 50; // Number of samples to use for long term time window - private int numLongSamples = 1000; + private int loadManagerNumberOfSamplesLongTermWindow = 1000; // How often in seconds to update the broker data - private long brokerDataUpdateIntervalSeconds = 60; + private long loadManagerBrokerDataUpdateIntervalInSeconds = 60; // How often in seconds to update the bundle data - private long bundleDataUpdateIntervalSeconds = 60; + private long loadManagerBundleDataUpdateIntervalInSeconds = 60; // Default throughput in to assume for new bundles - private double defaultMsgThroughputIn = 50000; + private double loadManagerDefaultMessageThroughputIn = 50000; // Default throughput out to assume for new bundles - private double defaultMsgThroughputOut = 50000; + private double loadManagerDefaultMessageThroughputOut = 50000; // Default message rate in to assume for new bundles - private double defaultMsgRateIn = 50; + private double loadManagerDefaultMessageRateIn = 50; // Default message rate out to assume for new bundles - private double defaultMsgRateOut = 50; + private double loadManagerDefaultMessageRateOut = 50; // Name of load manager to use - private String loadManagerName = "SimpleLoadManager"; + private String loadManagerClassName = "com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl"; // Name of placement strategy to use for new loadbalancer API. - private String newPlacementStrategyName = "LeastLongTermMessageRate"; + private String modularPlacementStrategyClassName = + "com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate"; - public int getNumShortSamples() { - return numShortSamples; + public int getLoadManagerNumberOfSamplesShortTermWindow() { + return loadManagerNumberOfSamplesShortTermWindow; } - public void setNumShortSamples(int numShortSamples) { - this.numShortSamples = numShortSamples; + public void setLoadManagerNumberOfSamplesShortTermWindow(int loadManagerNumberOfSamplesShortTermWindow) { + this.loadManagerNumberOfSamplesShortTermWindow = loadManagerNumberOfSamplesShortTermWindow; } - public int getNumLongSamples() { - return numLongSamples; + public int getLoadManagerNumberOfSamplesLongTermWindow() { + return loadManagerNumberOfSamplesLongTermWindow; } - public void setNumLongSamples(int numLongSamples) { - this.numLongSamples = numLongSamples; + public void setLoadManagerNumberOfSamplesLongTermWindow(int loadManagerNumberOfSamplesLongTermWindow) { + this.loadManagerNumberOfSamplesLongTermWindow = loadManagerNumberOfSamplesLongTermWindow; } - public long getBrokerDataUpdateIntervalSeconds() { - return brokerDataUpdateIntervalSeconds; + public long getLoadManagerBrokerDataUpdateIntervalInSeconds() { + return loadManagerBrokerDataUpdateIntervalInSeconds; } - public void setBrokerDataUpdateIntervalSeconds(long brokerDataUpdateIntervalSeconds) { - this.brokerDataUpdateIntervalSeconds = brokerDataUpdateIntervalSeconds; + public void setLoadManagerBrokerDataUpdateIntervalInSeconds(long loadManagerBrokerDataUpdateIntervalInSeconds) { + this.loadManagerBrokerDataUpdateIntervalInSeconds = loadManagerBrokerDataUpdateIntervalInSeconds; } - public long getBundleDataUpdateIntervalSeconds() { - return bundleDataUpdateIntervalSeconds; + public long getLoadManagerBundleDataUpdateIntervalInSeconds() { + return loadManagerBundleDataUpdateIntervalInSeconds; } - public void setBundleDataUpdateIntervalSeconds(long bundleDataUpdateIntervalSeconds) { - this.bundleDataUpdateIntervalSeconds = bundleDataUpdateIntervalSeconds; + public void setLoadManagerBundleDataUpdateIntervalInSeconds(long loadManagerBundleDataUpdateIntervalInSeconds) { + this.loadManagerBundleDataUpdateIntervalInSeconds = loadManagerBundleDataUpdateIntervalInSeconds; } - public double getDefaultMsgThroughputIn() { - return defaultMsgThroughputIn; + public double getLoadManagerDefaultMessageThroughputIn() { + return loadManagerDefaultMessageThroughputIn; } - public void setDefaultMsgThroughputIn(double defaultMsgThroughputIn) { - this.defaultMsgThroughputIn = defaultMsgThroughputIn; + public void setLoadManagerDefaultMessageThroughputIn(double loadManagerDefaultMessageThroughputIn) { + this.loadManagerDefaultMessageThroughputIn = loadManagerDefaultMessageThroughputIn; } - public double getDefaultMsgThroughputOut() { - return defaultMsgThroughputOut; + public double getLoadManagerDefaultMessageThroughputOut() { + return loadManagerDefaultMessageThroughputOut; } - public void setDefaultMsgThroughputOut(double defaultMsgThroughputOut) { - this.defaultMsgThroughputOut = defaultMsgThroughputOut; + public void setLoadManagerDefaultMessageThroughputOut(double loadManagerDefaultMessageThroughputOut) { + this.loadManagerDefaultMessageThroughputOut = loadManagerDefaultMessageThroughputOut; } - public double getDefaultMsgRateIn() { - return defaultMsgRateIn; + public double getLoadManagerDefaultMessageRateIn() { + return loadManagerDefaultMessageRateIn; } - public void setDefaultMsgRateIn(double defaultMsgRateIn) { - this.defaultMsgRateIn = defaultMsgRateIn; + public void setLoadManagerDefaultMessageRateIn(double loadManagerDefaultMessageRateIn) { + this.loadManagerDefaultMessageRateIn = loadManagerDefaultMessageRateIn; } - public double getDefaultMsgRateOut() { - return defaultMsgRateOut; + public double getLoadManagerDefaultMessageRateOut() { + return loadManagerDefaultMessageRateOut; } - public void setDefaultMsgRateOut(double defaultMsgRateOut) { - this.defaultMsgRateOut = defaultMsgRateOut; + public void setLoadManagerDefaultMessageRateOut(double loadManagerDefaultMessageRateOut) { + this.loadManagerDefaultMessageRateOut = loadManagerDefaultMessageRateOut; } - public String getLoadManagerName() { - return loadManagerName; + public String getLoadManagerClassName() { + return loadManagerClassName; } - public void setLoadManagerName(String loadManagerName) { - this.loadManagerName = loadManagerName; + public void setLoadManagerClassName(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; } - public String getNewPlacementStrategyName() { - return newPlacementStrategyName; + public String getModularPlacementStrategyClassName() { + return modularPlacementStrategyClassName; } - public void setNewPlacementStrategyName(String newPlacementStrategyName) { - this.newPlacementStrategyName = newPlacementStrategyName; + public void setModularPlacementStrategyClassName(String modularPlacementStrategyClassName) { + this.modularPlacementStrategyClassName = modularPlacementStrategyClassName; } } 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 index c5dc844e17f79..e85a9f5624ae9 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java @@ -4,48 +4,51 @@ 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). + * 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; - } + 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 index 81814fd6d86f3..c45d56acdb9f7 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java @@ -3,64 +3,77 @@ import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; /** - * Data class comprising the short term and long term historical data for this bundle. + * 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; + // 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; + // 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(){} + // 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 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); - } + /** + * 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); - } + /** + * 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 TimeAverageMessageData getShortTermData() { + return shortTermData; + } - public void setShortTermData(TimeAverageMessageData shortTermData) { - this.shortTermData = shortTermData; - } + public void setShortTermData(TimeAverageMessageData shortTermData) { + this.shortTermData = shortTermData; + } - public TimeAverageMessageData getLongTermData() { - return longTermData; - } + public TimeAverageMessageData getLongTermData() { + return longTermData; + } - public void setLongTermData(TimeAverageMessageData longTermData) { - this.longTermData = 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 index 0079d66ed6a1a..1b23daa4232cc 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java @@ -9,23 +9,25 @@ */ 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 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); - } + /** + * 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 index b742595f85bea..bcb1aa1843187 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -1,289 +1,302 @@ 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; -import java.util.*; - /** * 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; - - // 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(); - 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; - } - - // 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 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 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; - } + // 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; + + // 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(); + 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; + } + + // 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 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 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 319bbb1bb2cce..eaef1d027e5c8 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,7 +19,6 @@ 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; @@ -212,8 +211,8 @@ public void process(final WatchedEvent event) { final String newLoadManagerName = new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, this, null)); - config.setLoadManagerName(newLoadManagerName); - final LoadManager newLoadManager = LoadManager.create(config, PulsarService.this); + config.setLoadManagerClassName(newLoadManagerName); + final LoadManager newLoadManager = LoadManager.create(PulsarService.this); LOG.info("Created load manager: {}", newLoadManagerName); loadManager.get().disableBroker(); newLoadManager.start(); @@ -261,9 +260,9 @@ public void start() throws PulsarServerException { // Start load management service (even if load balancing is disabled) if (getZkClient().exists(DYNAMIC_LOAD_MANAGER_ZPATH, false) != null) { - config.setLoadManagerName(new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, false, null))); + config.setLoadManagerClassName(new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, false, null))); } - this.loadManager = new AtomicReference<>(LoadManager.create(config, this)); + this.loadManager = new AtomicReference<>(LoadManager.create(this)); this.startLoadManagementService(); @@ -339,7 +338,7 @@ public synchronized void brokerIsAFollowerNow() { try { ZkUtils.createFullPathOptimistic(getZkClient(), DYNAMIC_LOAD_MANAGER_ZPATH, - config.getLoadManagerName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + config.getLoadManagerClassName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } catch (KeeperException.NodeExistsException e) { // Ignore } 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 index b6f216b342ca8..dec4a88532052 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java @@ -1,144 +1,156 @@ package com.yahoo.pulsar.broker; -import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; - 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. + * 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; - } + 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 index a8f9052b875b1..78fa16d5e2770 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java @@ -6,125 +6,144 @@ * 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; - } + // 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 e0c99ace79dcd..21e3961669b18 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().get()).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().get()); - return lm.getResourceAvailabilityFor(ns).asMap(); + LoadManager lm = pulsar().getLoadManager().get(); + if (lm instanceof SimpleLoadManagerImpl) { + return ((SimpleLoadManagerImpl) lm).getResourceAvailabilityFor(ns).asMap(); + } else { + return Collections.emptyMap(); + } } 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/loadbalance/BrokerFilter.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java index b2633d861351f..804f31d2e48dd 100644 --- 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 @@ -1,30 +1,33 @@ package com.yahoo.pulsar.broker.loadbalance; +import java.util.Set; + import com.yahoo.pulsar.broker.BundleData; import com.yahoo.pulsar.broker.ServiceConfiguration; -import java.util.Set; - /** - * 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. + * 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); - - static BrokerFilter create(final ServiceConfiguration conf) { - // TODO - return null; - } + /** + * 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 index 4aa3289606682..069676e52af1e 100644 --- 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 @@ -1,38 +1,39 @@ package com.yahoo.pulsar.broker.loadbalance; -import com.yahoo.pulsar.broker.BrokerData; -import com.yahoo.pulsar.broker.BundleData; - 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. + * 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 broker names to their available data. + */ + private final Map brokerData; - /** - * Map from bundle names to their time-sensitive aggregated data. - */ - private final Map bundleData; + /** + * 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<>(); - } + /** + * Initialize a LoadData. + */ + public LoadData() { + this.brokerData = new ConcurrentHashMap<>(); + this.bundleData = new ConcurrentHashMap<>(); + } - public Map getBrokerData() { - return brokerData; - } + public Map getBrokerData() { + return brokerData; + } - public Map getBundleData() { - return bundleData; - } + 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 578109e5b3ebf..3a910d4edf381 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 @@ -20,91 +20,103 @@ 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.NewLoadManagerImpl; -import com.yahoo.pulsar.broker.loadbalance.impl.NewLoadManagerWrapper; +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; /** - * LoadManager runs though set of load reports collected from different brokers and generates a recommendation of - * namespace/ServiceUnit placement on machines/ResourceUnit. Each Concrete Load Manager will use different algorithms to - * generate this mapping. + * LoadManager runs though set of load reports collected from different brokers + * and generates a recommendation of namespace/ServiceUnit placement on + * machines/ResourceUnit. Each Concrete Load Manager will use different + * algorithms to generate this mapping. * - * Concrete Load Manager is also return the least loaded broker that should own the new namespace. + * Concrete Load Manager is also return the least loaded broker that should own + * the new namespace. */ public interface LoadManager { - public void start() throws PulsarServerException; - - /** - * Is centralized decision making to assign a new bundle. - */ - boolean isCentralized(); - - /** - * Returns the Least Loaded Resource Unit decided by some algorithm or criteria which is implementation specific - */ - ResourceUnit getLeastLoaded(ServiceUnitId su) throws Exception; - - /** - * Generate the load report - */ - LoadReport generateLoadReport() throws Exception; - - /** - * Set flag to force load report update - */ - void setLoadReportForceUpdateFlag(); - - /** - * Publish the current load report on ZK - */ - void writeLoadReportOnZookeeper() throws Exception; - - /** - * Update namespace bundle resource quota on ZK - */ - void writeResourceQuotasToZooKeeper() throws Exception; - - /** - * Generate load balancing stats metrics - */ - List getLoadBalancingMetrics(); - - /** - * Unload a candidate service unit to balance the load - */ - void doLoadShedding(); - - /** - * Namespace bundle split - */ - void doNamespaceBundleSplit() throws Exception; - - /** - * Determine the broker root. - */ - String getBrokerRoot(); - - /** - * Removes visibility of current broker from loadbalancer list so, other brokers can't redirect any request to this - * broker and this broker won't accept new connection requests. - * - * @throws Exception - */ - public void disableBroker() throws Exception; - - public void stop() throws PulsarServerException; - - static LoadManager create(final ServiceConfiguration conf, final PulsarService pulsar) { - switch (conf.getLoadManagerName()) { - case "NewLoadManager": - return new NewLoadManagerWrapper(new NewLoadManagerImpl(pulsar)); - case "SimpleLoadManager": - default: - return new SimpleLoadManagerImpl(pulsar); - } - } + public void start() throws PulsarServerException; + + /** + * Is centralized decision making to assign a new bundle. + */ + boolean isCentralized(); + + /** + * Returns the Least Loaded Resource Unit decided by some algorithm or + * criteria which is implementation specific + */ + ResourceUnit getLeastLoaded(ServiceUnitId su) throws Exception; + + /** + * Generate the load report + */ + LoadReport generateLoadReport() throws Exception; + + /** + * Set flag to force load report update + */ + void setLoadReportForceUpdateFlag(); + + /** + * Publish the current load report on ZK + */ + void writeLoadReportOnZookeeper() throws Exception; + + /** + * Update namespace bundle resource quota on ZK + */ + void writeResourceQuotasToZooKeeper() throws Exception; + + /** + * Generate load balancing stats metrics + */ + List getLoadBalancingMetrics(); + + /** + * Unload a candidate service unit to balance the load + */ + void doLoadShedding(); + + /** + * Namespace bundle split + */ + void doNamespaceBundleSplit() throws Exception; + + /** + * Determine the broker root. + */ + String getBrokerRoot(); + + /** + * Removes visibility of current broker from loadbalancer list so, other + * brokers can't redirect any request to this broker and this broker won't + * accept new connection requests. + * + * @throws Exception + */ + public void disableBroker() throws Exception; + + public void stop() throws PulsarServerException; + + 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.getConstructor(PulsarService.class).newInstance(pulsar); + if (loadManagerInstance instanceof LoadManager) { + return (LoadManager) loadManagerInstance; + } else if (loadManagerInstance instanceof ModularLoadManager) { + return new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance); + } + } catch (Exception e) { + // Ignore + } + // 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/LoadSheddingStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java index 1d8238297f6d3..4e073ee6fb112 100644 --- 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 @@ -1,30 +1,24 @@ package com.yahoo.pulsar.broker.loadbalance; -import com.yahoo.pulsar.broker.ServiceConfiguration; - import java.util.Map; -import java.util.Set; + +import com.yahoo.pulsar.broker.ServiceConfiguration; /** - * Load management component which determines the criteria for unloading bundles. + * 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 selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); - - /** - * Create a LoadSheddingStrategy from the given configuration. - * @param conf The configuration to create the strategy from. - * @return The created strategy. - */ - static LoadSheddingStrategy create(final ServiceConfiguration conf) { - // TODO - return null; - } + /** + * 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 selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java deleted file mode 100644 index 018cdef58b807..0000000000000 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewLoadManager.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.yahoo.pulsar.broker.loadbalance; - -import com.yahoo.pulsar.broker.PulsarServerException; - -/** - * 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 NewLoadManager { - - /** - * 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(); - - /** - * Get the broker root ZooKeeper path. - */ - String getBrokerRoot(); - - /** - * As the leader broker, find a suitable broker for the assignment of the given bundle. - * @param bundleToAssign Full name of the bundle to assign. - * @return The name of the selected broker, as it appears on ZooKeeper. - */ - String selectBrokerForAssignment(String bundleToAssign); - - /** - * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. - */ - void updateLocalBrokerData(); - - /** - * 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, 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/NewPlacementStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java deleted file mode 100644 index ea1f1135425f8..0000000000000 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/NewPlacementStrategy.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.yahoo.pulsar.broker.loadbalance; - -import com.yahoo.pulsar.broker.BundleData; -import com.yahoo.pulsar.broker.ServiceConfiguration; -import com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; - -import java.util.Set; - -/** - * Interface which serves as a component for NewLoadManagerImpl, flexibly allowing the injection of potentially - * complex strategies. - */ -public interface NewPlacementStrategy { - - /** - * 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 NewPlacementStrategy create(final ServiceConfiguration conf) { - switch (conf.getNewPlacementStrategyName()) { - case "LeastLongTermMessageRate": - default: - return new LeastLongTermMessageRate(); - } - } -} 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 index 69d32e2160523..df5692c45fb03 100644 --- 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 @@ -1,119 +1,135 @@ package com.yahoo.pulsar.broker.loadbalance.impl; -import com.yahoo.pulsar.broker.*; -import com.yahoo.pulsar.broker.loadbalance.LoadData; -import com.yahoo.pulsar.broker.loadbalance.LoadSheddingStrategy; +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 java.util.*; +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. + * An abstract class which makes a LoadSheddingStrategy which makes decisions + * based on standard deviation easier to implement. */ 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; + // 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<>(); - } + /** + * 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 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); + // 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); + // 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 obsessive 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 selectBundlesForUnloading(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(); + /** + * Recommend that all of the returned bundles be unloaded based on observing + * obsessive 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 selectBundlesForUnloading(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(); + // 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(); + 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; - } + 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 index 2730d81703d84..186093e468fcc 100644 --- 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 @@ -1,69 +1,80 @@ package com.yahoo.pulsar.broker.loadbalance.impl; -import com.yahoo.pulsar.broker.*; -import com.yahoo.pulsar.broker.loadbalance.LoadData; -import com.yahoo.pulsar.broker.loadbalance.NewPlacementStrategy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; -import java.util.Map; 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. + * Placement strategy which selects a broker based on which one has the least + * long term message rate. */ -public class LeastLongTermMessageRate implements NewPlacementStrategy { - private static Logger log = LoggerFactory.getLogger(LeastLongTermMessageRate.class); +public class LeastLongTermMessageRate implements ModularLoadManagerStrategy { + private static Logger log = LoggerFactory.getLogger(LeastLongTermMessageRate.class); - // Maintain this list to reduce object creation. - private ArrayList bestBrokers; + // Maintain this list to reduce object creation. + private ArrayList bestBrokers; - public LeastLongTermMessageRate() { - bestBrokers = new ArrayList<>(); - } + public LeastLongTermMessageRate(final ServiceConfiguration conf) { + bestBrokers = new ArrayList<>(); + } - // Form a score for a broker using its preallocated bundle data and time average data. - private static double getScore(final BrokerData brokerData) { - double totalMessageRate = 0; - for (BundleData bundleData: brokerData.getPreallocatedBundleData().values()) { - final TimeAverageMessageData longTermData = bundleData.getLongTermData(); - totalMessageRate += longTermData.getMsgRateIn() + longTermData.getMsgRateOut(); - } - final TimeAverageBrokerData timeAverageData = brokerData.getTimeAverageData(); - return totalMessageRate + timeAverageData.getLongTermMsgRateIn() + timeAverageData.getLongTermMsgRateOut(); - } + // Form a score for a broker using its preallocated bundle data and time + // average data. + private static double getScore(final BrokerData brokerData) { + double totalMessageRate = 0; + for (BundleData bundleData : brokerData.getPreallocatedBundleData().values()) { + final TimeAverageMessageData longTermData = bundleData.getLongTermData(); + totalMessageRate += longTermData.getMsgRateIn() + longTermData.getMsgRateOut(); + } + final TimeAverageBrokerData timeAverageData = brokerData.getTimeAverageData(); + return totalMessageRate + timeAverageData.getLongTermMsgRateIn() + timeAverageData.getLongTermMsgRateOut(); + } - /** - * 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)); - 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); - } - } - return bestBrokers.get(ThreadLocalRandom.current().nextInt(bestBrokers.size())); - } + /** + * 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)); + 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); + } + } + return bestBrokers.get(ThreadLocalRandom.current().nextInt(bestBrokers.size())); + } } diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java deleted file mode 100644 index cf7a79af9d570..0000000000000 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerImpl.java +++ /dev/null @@ -1,524 +0,0 @@ -package com.yahoo.pulsar.broker.loadbalance.impl; - -import com.google.common.cache.*; -import com.yahoo.pulsar.broker.*; -import com.yahoo.pulsar.broker.loadbalance.*; -import com.yahoo.pulsar.client.admin.PulsarAdmin; -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; -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 java.io.IOException; -import java.net.URL; -import java.util.*; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import static com.google.common.base.Preconditions.checkArgument; - -public class NewLoadManagerImpl implements NewLoadManager, ZooKeeperCacheListener { - public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; - public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; - public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; - - private static final int MIBI = 1024 * 1024; - private static final Logger log = LoggerFactory.getLogger(NewLoadManagerImpl.class); - - private final LocalBrokerData localData; - private final LoadData loadData; - - // Used to determine whether a bundle is preallocated. - private final Map preallocatedBundleToBroker; - - // Set of broker candidates to reuse so that object creation is avoided. - private final Set brokerCandidateCache; - - // Used to filter brokers from being selected for assignment. - private final List filterPipeline; - - // Pipeline used to determine what namespaces, if any, should be unloaded. - private final List loadSheddingPipeline; - - // Strategy used to determine where new topics should be placed. - private final NewPlacementStrategy placementStrategy; - - private final PulsarService pulsar; - private final ZooKeeper zkClient; - private final ServiceConfiguration conf; - private final BrokerHostUsage brokerHostUsage; - private final ZooKeeperDataCache brokerDataCache; - private final ZooKeeperChildrenCache availableActiveBrokers; - private final ScheduledExecutorService scheduler; - private final LoadingCache adminCache; - - // 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; - - // Timestamp of last invocation of updateBundleData. - private long lastBundleDataUpdate; - - private String brokerZnodePath; - private final String brokerRoot; - - // System resource usage directly after starting. - private SystemResourceUsage baselineSystemResourceUsage; - - public NewLoadManagerImpl(final PulsarService pulsar) { - this(pulsar, LOADBALANCE_BROKERS_ROOT); - } - - /** - * Initialize this load manager. - * @param pulsar Client to construct this manager from. - * @param brokerRoot ZooKeeper path containing some data implementing ServiceLookup. - */ - public NewLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { - this.brokerRoot = brokerRoot; - this.pulsar = pulsar; - zkClient = pulsar.getZkClient(); - conf = pulsar.getConfiguration(); - - loadData = new LoadData(); - preallocatedBundleToBroker = new ConcurrentHashMap<>(); - brokerCandidateCache = new HashSet<>(); - filterPipeline = new ArrayList<>(); - loadSheddingPipeline = new ArrayList<>(); - localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); - placementStrategy = NewPlacementStrategy.create(conf); - 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()); - } - }); - - - // Initialize the default - defaultStats = new NamespaceBundleStats(); - defaultStats.msgThroughputIn = conf.getDefaultMsgThroughputIn(); - defaultStats.msgThroughputOut = conf.getDefaultMsgThroughputOut(); - defaultStats.msgRateIn = conf.getDefaultMsgRateIn(); - defaultStats.msgRateOut = conf.getDefaultMsgRateOut(); - if (SystemUtils.IS_OS_LINUX) { - brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); - } else { - brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); - } - - 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); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); - 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(NewLoadManagerImpl.this::updateAll); - } - }); - scheduler = Executors.newScheduledThreadPool(1); - } - - /* - * 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", brokerRoot, broker); - final LocalBrokerData localData = brokerDataCache.get(key) - .orElseThrow(KeeperException.NoNodeException::new); - - if (brokerDataMap.containsKey(broker)) { - // Replace previous local broker data. - loadData.getBrokerData().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); - } - } - - /* - * Use the Pulsar client to acquire the namespace bundle stats. - */ - private Map getBundleStats() { - return pulsar.getBrokerService().getBundleStats(); - } - - /** - * Update both the broker data and the bundle data. - */ - public void updateAll() { - updateAllBrokerData(); - updateBundleData(); - } - - /** - * 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. - */ - public 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); - } - } - - // 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() + conf.getBrokerDataUpdateIntervalSeconds() * 1000; - } - - // Determine if the bundle data requires an update by measuring the time past since the last update. - private boolean needBundleDataUpdate() { - return System.currentTimeMillis() > lastBundleDataUpdate + conf.getBundleDataUpdateIntervalSeconds() * 1000; - } - - // 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. - } - } - } - - // Get the ZooKeeper path for the given bundle full name. - public static String getBundleDataZooKeeperPath(final String bundle) { - return BUNDLE_DATA_ZPATH + "/" + bundle; - } - - // Get the total number of used bytes in the JVM. - private static long getRealtimeJVMHeapUsageBytes() { - return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); - } - - // Get the system resource usage for this broker. - 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(); - long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); - 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; - } - - // 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); - } - - // 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(conf.getNumShortSamples(), conf.getNumLongSamples(), defaultStats); - } - return bundleData; - } - - /** - * 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); - } - } - } - - 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()); - } - - /** - * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new - * brokers. - */ - @Override - public void doLoadShedding() { - for (LoadSheddingStrategy strategy: loadSheddingPipeline) { - final Map bundlesToUnload = strategy.selectBundlesForUnloading(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( - getNamespaceNameFromBundleName(bundle), 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? - } - - @Override - public String getBrokerRoot() { - return brokerRoot; - } - - /** - * 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 bundle Full name of the bundle to assign. - * @return The name of the selected broker, as it appears on ZooKeeper. - */ - @Override - public synchronized String selectBrokerForAssignment(final String bundle) { - // ?: Is it too inefficient to make this synchronized? If so, it may be a good idea to use weighted random - // or atomic data. - 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(); - brokerCandidateCache.addAll(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, brokerRoot); - - String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); - brokerZnodePath = brokerRoot + "/" + 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(); - baselineSystemResourceUsage = getSystemResourceUsage(); - } 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 = getSystemResourceUsage(); - 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/NewLoadManagerWrapper.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java deleted file mode 100644 index f777eabf9b242..0000000000000 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/NewLoadManagerWrapper.java +++ /dev/null @@ -1,91 +0,0 @@ -package com.yahoo.pulsar.broker.loadbalance.impl; - -import com.yahoo.pulsar.broker.PulsarServerException; -import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; -import com.yahoo.pulsar.broker.loadbalance.NewLoadManager; -import com.yahoo.pulsar.broker.loadbalance.LoadManager; -import com.yahoo.pulsar.broker.loadbalance.ResourceUnit; -import com.yahoo.pulsar.broker.stats.Metrics; -import com.yahoo.pulsar.common.naming.ServiceUnitId; - -import java.util.Collections; -import java.util.List; - -/** - * Wrapper class allowing classes of instance NewLoadManager to be compatible with the interface LoadManager. - */ -public class NewLoadManagerWrapper implements LoadManager { - private NewLoadManager loadManager; - - public NewLoadManagerWrapper(final NewLoadManager 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 String getBrokerRoot() { - return loadManager.getBrokerRoot(); - } - - @Override - public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { - return new SimpleResourceUnit(String.format("http://%s", - loadManager.selectBrokerForAssignment(serviceUnit.toString())), - new PulsarResourceDescription()); - } - - @Override - public List getLoadBalancingMetrics() { - return Collections.emptyList(); - } - - @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 520d6fe84d414..0fa8864232ddf 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 @@ -259,11 +259,9 @@ public void start() throws PulsarServerException { // ignore the exception, node might be present already } } - Thread.sleep(5000); String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); brokerZnodePath = brokerRoot + "/" + lookupServiceAddress; LoadReport loadReport = null; - Thread.sleep(5000); try { loadReport = generateLoadReport(); this.lastResourceUsageTimestamp = loadReport.getTimestamp(); 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 index 31e6b7d029e53..45cca44392e02 100644 --- 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 @@ -2,11 +2,11 @@ // For backwards compatibility purposes. public interface ServiceLookupData { - public String getWebServiceUrl(); + public String getWebServiceUrl(); - public String getWebServiceUrlTls(); + public String getWebServiceUrlTls(); - public String getPulsarServiceUrl(); + public String getPulsarServiceUrl(); - public String getPulsarServiceUrlTls(); + public String getPulsarServiceUrlTls(); } diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java deleted file mode 100644 index 69596a7a61b37..0000000000000 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/BrokerMonitor.java +++ /dev/null @@ -1,186 +0,0 @@ -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 java.io.BufferedReader; -import java.io.InputStreamReader; -import java.net.ServerSocket; -import java.net.Socket; -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 BrokerMonitor { - 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)) { - System.out.println("Lost broker: " + oldBroker); - } - } - for (String newBroker: newBrokers) { - if (!brokers.contains(newBroker)) { - System.out.println("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(); - - System.out.println("\nLoad Report for " + brokerName + ":"); - System.out.println("---------------"); - - - System.out.println("\nNum Topics: " + loadReport.getNumTopics()); - System.out.println("Num Bundles: " + loadReport.getNumBundles()); - - System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); - System.out.println(String.format("Allocated CPU: %.2f%%", - percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); - System.out.println(String.format("Preallocated CPU: %.2f%%", - percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); - - System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); - System.out.println(String.format("Allocated Memory: %.2f%%", - percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); - System.out.println(String.format("Preallocated Memory: %.2f%%", - percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); - - System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); - System.out.println(String.format("Allocated Bandwidth In: %.2f%%", - percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); - System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", - percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); - - System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); - System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", - percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); - System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", - percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); - - System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); - - System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); - System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); - System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); - System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); - System.out.println(); - if (!loadReport.getBundleGains().isEmpty()) { - for (String bundle: loadReport.getBundleGains()) { - System.out.println("Gained Bundle: " + bundle); - } - System.out.println(); - } - if (!loadReport.getBundleLosses().isEmpty()) { - for (String bundle: loadReport.getBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); - } - System.out.println(); - } - } - } - - static class Arguments { - @Parameter(names = {"--connect-string"}, description = "Zookeeper connect string", required = true) - public String connectString = null; - } - - public BrokerMonitor(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 BrokerMonitor monitor = new BrokerMonitor(zkClient); - monitor.start(); - } 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 index 5fd1c4926d5c6..b2b9fd41c4420 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -1,5 +1,26 @@ 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; @@ -8,603 +29,621 @@ 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.apache.bookkeeper.util.ZkUtils; -import org.apache.zookeeper.*; - -import java.io.*; -import java.net.Socket; -import java.nio.file.Paths; -import java.util.*; /** - * 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. + * 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 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]; - System.out.format("Found %d servers:\n", 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()); - System.out.format("Connected to %s\n", 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) { - System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", - 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) { - System.out.println("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); - System.out.println("Sending trade request to " + servers[i]); - outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); - writeProducerOptions(outputStreams[i], arguments, destination); - outputStreams[i].flush(); - if (inputStreams[i].read() != -1) { - System.out.println("Created producer and consumer for " + destination); - } else { - System.out.format("ERROR: Socket to %s closed\n", 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 { - System.out.println("Searching for server with topic " + destination); - for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); - foundTopic = true; - break; - case LoadSimulationServer.NO_SUCH_TOPIC: - break; - case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - break; - default: - System.out.println("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)) { - System.out.format("ERROR: Topic %s not found\n", 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)); - System.out.println("Searching for server with topic " + destination); - for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); - foundTopic = true; - break; - case LoadSimulationServer.NO_SUCH_TOPIC: - break; - case LoadSimulationServer.REDUNDANT_COMMAND: - System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); - foundTopic = true; - break; - case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - break; - default: - System.out.println("ERROR: Unknown response signal received: " + readValue); - } - } - if (!foundTopic) { - System.out.format("ERROR: Topic %s not found\n", 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(LoadSimulationServer.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) { - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - } else if (foundOnServer == 0) { - System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, - servers[i]); - } else if (foundOnServer > 0){ - System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, - tenant, group, servers[i]); - numFound += foundOnServer; - } else { - System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, - servers[i]); - } - } - if (numFound == 0) { - System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); - } else { - System.out.format("Found %d topics belonging to tenant %s and group %s\n", 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(LoadSimulationServer.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); - System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); - ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - System.out.format("Creating new API data at %s\n", 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: - System.out.format("ERROR: Unknown command \"%s\"\n", 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(); - } + 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]; + System.out.format("Found %d servers:\n", 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()); + System.out.format("Connected to %s\n", 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) { + System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", 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) { + System.out.println("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); + System.out.println("Sending trade request to " + servers[i]); + outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); + writeProducerOptions(outputStreams[i], arguments, destination); + outputStreams[i].flush(); + if (inputStreams[i].read() != -1) { + System.out.println("Created producer and consumer for " + destination); + } else { + System.out.format("ERROR: Socket to %s closed\n", 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 { + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("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)) { + System.out.format("ERROR: Topic %s not found\n", 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)); + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case LoadSimulationServer.REDUNDANT_COMMAND: + System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); + foundTopic = true; + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("ERROR: Unknown response signal received: " + readValue); + } + } + if (!foundTopic) { + System.out.format("ERROR: Topic %s not found\n", 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(LoadSimulationServer.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) { + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + } else if (foundOnServer == 0) { + System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, + servers[i]); + } else if (foundOnServer > 0) { + System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, tenant, + group, servers[i]); + numFound += foundOnServer; + } else { + System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, + servers[i]); + } + } + if (numFound == 0) { + System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); + } else { + System.out.format("Found %d topics belonging to tenant %s and group %s\n", 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(LoadSimulationServer.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); + System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); + ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + System.out.format("Creating new API data at %s\n", 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: + System.out.format("ERROR: Unknown command \"%s\"\n", 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/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index 79dfb956d1aee..09503acfe9470 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -1,330 +1,357 @@ 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.*; +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.apache.commons.lang.SystemUtils; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.net.ServerSocket; -import java.net.Socket; -import java.util.Map; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; /** - * LoadSimulationServer is used to simulate client load by maintaining producers and consumers for topics. - * Instances of this class are controlled across a network via LoadSimulationController. + * LoadSimulationServer 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 LoadSimulationServer { - // 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.createProducerAsync(topic, producerConf).get(); - } 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 LoadSimulationServer. - 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 LoadSimulationServer(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(); - - // Disable connection pooling. - clientConf.setConnectionsPerBroker(0); - - // 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 LoadSimulationServer(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. - System.out.println("Listening for controller command..."); - final Socket socket = serverSocket.accept(); - System.out.format("Connected to %s\n", socket.getInetAddress().getHostName()); - executor.submit(() -> { - try { - handle(socket); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - }); - } - } + // 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.createProducerAsync(topic, producerConf).get(); + } 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 LoadSimulationServer. + 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 LoadSimulationServer(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(); + + // Disable connection pooling. + clientConf.setConnectionsPerBroker(0); + + // 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 LoadSimulationServer(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. + System.out.println("Listening for controller command..."); + final Socket socket = serverSocket.accept(); + System.out.format("Connected to %s\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/NewBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java deleted file mode 100644 index 9fcc8048637b6..0000000000000 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/NewBrokerMonitor.java +++ /dev/null @@ -1,193 +0,0 @@ -package com.yahoo.pulsar.testclient; - -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.NewLoadManagerImpl; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; - -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.net.ServerSocket; -import java.net.Socket; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -public class NewBrokerMonitor { - private static final String BROKER_ROOT = "/loadbalance/new-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.EMPTY_SET; - } - - 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)) { - System.out.println("Lost broker: " + oldBroker); - } - } - for (String newBroker: newBrokers) { - if (!brokers.contains(newBroker)) { - System.out.println("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) { - System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); - System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); - System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); - System.out.format("Message Rate Out: %.2f msgs/s\n", msgRateOut); - } - - public synchronized void printBrokerData(final String brokerPath) { - final String broker = brokerNameFromPath(brokerPath); - final String timeAveragePath = NewLoadManagerImpl.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); - } - - System.out.println("\nBroker Data for " + broker + ":"); - System.out.println("---------------"); - - - System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); - System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); - System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); - System.out.println("Num Producers: " + localBrokerData.getNumProducers()); - - System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); - - System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); - - System.out.println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); - - System.out.println("\nLatest Data:\n"); - 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); - } - System.out.println("\nShort Term Data:\n"); - printMessageData(timeAverageData.getShortTermMsgThroughputIn(), - timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), - timeAverageData.getShortTermMsgRateOut()); - - System.out.println("\nLong Term Data:\n"); - printMessageData(timeAverageData.getLongTermMsgThroughputIn(), - timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), - timeAverageData.getLongTermMsgRateOut()); - - - System.out.println(); - if (!localBrokerData.getLastBundleGains().isEmpty()) { - for (String bundle: localBrokerData.getLastBundleGains()) { - System.out.println("Gained Bundle: " + bundle); - } - System.out.println(); - } - if (!localBrokerData.getLastBundleLosses().isEmpty()) { - for (String bundle: localBrokerData.getLastBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); - } - System.out.println(); - } - } - } - - static class Arguments { - @Parameter(names = {"--connect-string"}, description = "Zookeeper connect string", required = true) - public String connectString = null; - } - - public NewBrokerMonitor(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 NewBrokerMonitor monitor = new NewBrokerMonitor(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 From 1f85e00bf1f0c7b2b76dc920cb9de285eb944e35 Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 22 Mar 2017 14:04:18 -0700 Subject: [PATCH 03/26] Fix imports --- .../main/java/com/yahoo/pulsar/broker/PulsarService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 eaef1d027e5c8..dc48aa94c3a1f 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 @@ -31,7 +31,12 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.util.OrderedSafeExecutor; import org.apache.bookkeeper.util.ZkUtils; -import org.apache.zookeeper.*; +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; import org.slf4j.LoggerFactory; From 713ff38a4fee9aaf0f516e369a2815863496c9aa Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 22 Mar 2017 14:21:17 -0700 Subject: [PATCH 04/26] Add back renamed classes --- .../loadbalance/ModularLoadManager.java | 81 +++ .../ModularLoadManagerStrategy.java | 51 ++ .../impl/ModularLoadManagerImpl.java | 579 ++++++++++++++++++ .../impl/ModularLoadManagerWrapper.java | 97 +++ .../ModularLoadManagerBrokerMonitor.java | 191 ++++++ .../SimpleLoadManagerBrokerMonitor.java | 182 ++++++ 6 files changed, 1181 insertions(+) create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java 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..ae10882f263b0 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java @@ -0,0 +1,81 @@ +package com.yahoo.pulsar.broker.loadbalance; + +import com.yahoo.pulsar.broker.PulsarServerException; + +/** + * 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(); + + /** + * Get the broker root ZooKeeper path. + */ + String getBrokerRoot(); + + /** + * As the leader broker, find a suitable broker for the assignment of the + * given bundle. + * + * @param bundleToAssign + * Full name of the bundle to assign. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + String selectBrokerForAssignment(String bundleToAssign); + + /** + * As any broker, retrieve the namespace bundle stats and system resource + * usage to update data local to this broker. + */ + void updateLocalBrokerData(); + + /** + * 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, 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..af5a7a71044fa --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java @@ -0,0 +1,51 @@ +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 { + final Class placementStrategyClass = Class.forName(conf.getModularPlacementStrategyClassName()); + + // Assume there is a constructor of one argument of + // ServiceConfiguration. + return (ModularLoadManagerStrategy) placementStrategyClass.getConstructor(ServiceConfiguration.class) + .newInstance(conf); + } catch (Exception e) { + // Ignore + } + return new LeastLongTermMessageRate(conf); + } +} 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..f5617fcf30a90 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -0,0 +1,579 @@ +package com.yahoo.pulsar.broker.loadbalance.impl; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.IOException; +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 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.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 { + public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; + public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; + public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; + + private static final int MIBI = 1024 * 1024; + private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); + + private final LocalBrokerData localData; + private final LoadData loadData; + + // Used to determine whether a bundle is preallocated. + private final Map preallocatedBundleToBroker; + + // Set of broker candidates to reuse so that object creation is avoided. + private final Set brokerCandidateCache; + + // Used to filter brokers from being selected for assignment. + private final List filterPipeline; + + // Pipeline used to determine what namespaces, if any, should be unloaded. + private final List loadSheddingPipeline; + + // Strategy used to determine where new topics should be placed. + private final ModularLoadManagerStrategy placementStrategy; + + private final SimpleResourceAllocationPolicies policies; + + private final PulsarService pulsar; + private final ZooKeeper zkClient; + private final ServiceConfiguration conf; + private final BrokerHostUsage brokerHostUsage; + private final ZooKeeperDataCache brokerDataCache; + private final ZooKeeperChildrenCache availableActiveBrokers; + private final ScheduledExecutorService scheduler; + private final LoadingCache adminCache; + + // 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; + + // Timestamp of last invocation of updateBundleData. + private long lastBundleDataUpdate; + + private String brokerZnodePath; + private final String brokerRoot; + + // System resource usage directly after starting. + private SystemResourceUsage baselineSystemResourceUsage; + + public ModularLoadManagerImpl(final PulsarService pulsar) { + this(pulsar, LOADBALANCE_BROKERS_ROOT); + } + + /** + * Initialize this load manager. + * + * @param pulsar + * Client to construct this manager from. + * @param brokerRoot + * ZooKeeper path containing some data implementing + * ServiceLookup. + */ + public ModularLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { + this.brokerRoot = brokerRoot; + this.pulsar = pulsar; + zkClient = pulsar.getZkClient(); + conf = pulsar.getConfiguration(); + policies = new SimpleResourceAllocationPolicies(pulsar); + loadData = new LoadData(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + brokerCandidateCache = new HashSet<>(); + filterPipeline = new ArrayList<>(); + loadSheddingPipeline = new ArrayList<>(); + localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + placementStrategy = ModularLoadManagerStrategy.create(conf); + 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()); + } + }); + + // Initialize the default + defaultStats = new NamespaceBundleStats(); + defaultStats.msgThroughputIn = conf.getLoadManagerDefaultMessageThroughputIn(); + defaultStats.msgThroughputOut = conf.getLoadManagerDefaultMessageThroughputOut(); + defaultStats.msgRateIn = conf.getLoadManagerDefaultMessageRateIn(); + defaultStats.msgRateOut = conf.getLoadManagerDefaultMessageRateOut(); + if (SystemUtils.IS_OS_LINUX) { + brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); + } else { + brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); + } + + 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); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); + 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); + } + }); + scheduler = Executors.newScheduledThreadPool(1); + } + + /* + * 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", brokerRoot, broker); + final LocalBrokerData localData = brokerDataCache.get(key) + .orElseThrow(KeeperException.NoNodeException::new); + + if (brokerDataMap.containsKey(broker)) { + // Replace previous local broker data. + loadData.getBrokerData().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); + } + } + + /* + * Use the Pulsar client to acquire the namespace bundle stats. + */ + private Map getBundleStats() { + return pulsar.getBrokerService().getBundleStats(); + } + + /** + * Update both the broker data and the bundle data. + */ + public void updateAll() { + updateAllBrokerData(); + updateBundleData(); + } + + /** + * 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. + */ + public 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); + } + } + + // 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() + + conf.getLoadManagerBrokerDataUpdateIntervalInSeconds() * 1000; + } + + // Determine if the bundle data requires an update by measuring the time + // past since the last update. + private boolean needBundleDataUpdate() { + return System.currentTimeMillis() > lastBundleDataUpdate + + conf.getLoadManagerBundleDataUpdateIntervalInSeconds() * 1000; + } + + // 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. + } + } + } + + // Get the ZooKeeper path for the given bundle full name. + public static String getBundleDataZooKeeperPath(final String bundle) { + return BUNDLE_DATA_ZPATH + "/" + bundle; + } + + // Get the total number of used bytes in the JVM. + private static long getRealtimeJVMHeapUsageBytes() { + return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + } + + // Get the system resource usage for this broker. + 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(); + long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); + 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; + } + + // 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); + } + + // 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(conf.getLoadManagerNumberOfSamplesShortTermWindow(), + conf.getLoadManagerNumberOfSamplesLongTermWindow(), defaultStats); + } + return bundleData; + } + + /** + * 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); + } + } + } + + 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()); + } + + /** + * As the leader broker, select bundles for the namespace service to unload + * so that they may be reassigned to new brokers. + */ + @Override + public void doLoadShedding() { + for (LoadSheddingStrategy strategy : loadSheddingPipeline) { + final Map bundlesToUnload = strategy.selectBundlesForUnloading(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( + getNamespaceNameFromBundleName(bundle), 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? + } + + @Override + public String getBrokerRoot() { + return brokerRoot; + } + + /** + * 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 bundle + * Full name of the bundle to assign. + * @return The name of the selected broker, as it appears on ZooKeeper. + */ + @Override + public synchronized String selectBrokerForAssignment(final String bundle) { + // ?: Is it too inefficient to make this synchronized? If so, it may be + // a good idea to use weighted random + // or atomic data. + 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(); + brokerCandidateCache.addAll(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, brokerRoot); + + String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); + brokerZnodePath = brokerRoot + "/" + 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(); + baselineSystemResourceUsage = getSystemResourceUsage(); + } 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 = getSystemResourceUsage(); + 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..b377285e98e32 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerWrapper.java @@ -0,0 +1,97 @@ +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 PulsarService pulsar) { + this(new ModularLoadManagerImpl(pulsar)); + } + + 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 String getBrokerRoot() { + return loadManager.getBrokerRoot(); + } + + @Override + public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { + return new SimpleResourceUnit( + String.format("http://%s", loadManager.selectBrokerForAssignment(serviceUnit.toString())), + new PulsarResourceDescription()); + } + + @Override + public List getLoadBalancingMetrics() { + return Collections.emptyList(); + } + + @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-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..b1318f6fb0aed --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java @@ -0,0 +1,191 @@ +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; + +public class ModularLoadManagerBrokerMonitor { + private static final String BROKER_ROOT = "/loadbalance/new-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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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) { + System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); + System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); + System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); + System.out.format("Message Rate Out: %.2f msgs/s\n", 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); + } + + System.out.println("\nBroker Data for " + broker + ":"); + System.out.println("---------------"); + + System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); + System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); + System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); + System.out.println("Num Producers: " + localBrokerData.getNumProducers()); + + System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); + + System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); + + System.out + .println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); + + System.out.println("\nLatest Data:\n"); + 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); + } + System.out.println("\nShort Term Data:\n"); + printMessageData(timeAverageData.getShortTermMsgThroughputIn(), + timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), + timeAverageData.getShortTermMsgRateOut()); + + System.out.println("\nLong Term Data:\n"); + printMessageData(timeAverageData.getLongTermMsgThroughputIn(), + timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), + timeAverageData.getLongTermMsgRateOut()); + + System.out.println(); + if (!localBrokerData.getLastBundleGains().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!localBrokerData.getLastBundleLosses().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + 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/SimpleLoadManagerBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java new file mode 100644 index 0000000000000..9108bd5530903 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java @@ -0,0 +1,182 @@ +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 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 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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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(); + + System.out.println("\nLoad Report for " + brokerName + ":"); + System.out.println("---------------"); + + System.out.println("\nNum Topics: " + loadReport.getNumTopics()); + System.out.println("Num Bundles: " + loadReport.getNumBundles()); + + System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); + System.out.println(String.format("Allocated CPU: %.2f%%", + percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); + System.out.println(String.format("Preallocated CPU: %.2f%%", + percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); + + System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); + System.out.println(String.format("Allocated Memory: %.2f%%", + percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); + System.out.println(String.format("Preallocated Memory: %.2f%%", + percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); + + System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); + System.out.println(String.format("Allocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + + System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); + System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + + System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); + + System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); + System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); + System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); + System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); + System.out.println(); + if (!loadReport.getBundleGains().isEmpty()) { + for (String bundle : loadReport.getBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!loadReport.getBundleLosses().isEmpty()) { + for (String bundle : loadReport.getBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + 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); + } + } +} From 5e343c9c1257f0056d8404e4a6f60ed7054d68fe Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 22 Mar 2017 15:13:46 -0700 Subject: [PATCH 05/26] Add policies to ModularLoadManagerImpl --- .../loadbalance/ModularLoadManager.java | 7 +- .../impl/ModularLoadManagerImpl.java | 86 ++++++++++++++++++- .../impl/ModularLoadManagerWrapper.java | 7 +- 3 files changed, 88 insertions(+), 12 deletions(-) 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 index ae10882f263b0..00cb3d7fc53d1 100644 --- 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 @@ -1,6 +1,7 @@ package com.yahoo.pulsar.broker.loadbalance; import com.yahoo.pulsar.broker.PulsarServerException; +import com.yahoo.pulsar.common.naming.ServiceUnitId; /** * New proposal for a load manager interface which attempts to use more @@ -38,11 +39,11 @@ public interface ModularLoadManager { * As the leader broker, find a suitable broker for the assignment of the * given bundle. * - * @param bundleToAssign - * Full name of the bundle to assign. + * @param serviceUnit + * ServiceUnitId for the bundle. * @return The name of the selected broker, as it appears on ZooKeeper. */ - String selectBrokerForAssignment(String bundleToAssign); + String selectBrokerForAssignment(ServiceUnitId serviceUnit); /** * As any broker, retrieve the namespace bundle stats and system resource 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 index f5617fcf30a90..ce8a4328611f7 100644 --- 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 @@ -3,6 +3,7 @@ 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; @@ -45,6 +46,8 @@ 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.NamespaceName; +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; @@ -68,6 +71,8 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach // Set of broker candidates to reuse so that object creation is avoided. private final Set brokerCandidateCache; + private final Set primariesCache; + private final Set sharedCache; // Used to filter brokers from being selected for assignment. private final List filterPipeline; @@ -124,6 +129,8 @@ public ModularLoadManagerImpl(final PulsarService pulsar, final String brokerRoo loadData = new LoadData(); preallocatedBundleToBroker = new ConcurrentHashMap<>(); brokerCandidateCache = new HashSet<>(); + primariesCache = new HashSet<>(); + sharedCache = new HashSet<>(); filterPipeline = new ArrayList<>(); loadSheddingPipeline = new ArrayList<>(); localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), @@ -434,15 +441,17 @@ public void onUpdate(final String path, final LocalBrokerData data, final Stat s * As the leader broker, find a suitable broker for the assignment of the * given bundle. * - * @param bundle - * Full name of the bundle to assign. + * @param serviceUnit + * ServiceUnitId for the bundle. * @return The name of the selected broker, as it appears on ZooKeeper. */ @Override - public synchronized String selectBrokerForAssignment(final String bundle) { + 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. @@ -451,6 +460,7 @@ public synchronized String selectBrokerForAssignment(final String bundle) { final BundleData data = loadData.getBundleData().computeIfAbsent(bundle, key -> getBundleDataOrDefault(bundle)); brokerCandidateCache.clear(); brokerCandidateCache.addAll(loadData.getBrokerData().keySet()); + policyFilter(serviceUnit); // Use the filter pipeline to finalize broker candidates. for (BrokerFilter filter : filterPipeline) { @@ -464,6 +474,76 @@ public synchronized String selectBrokerForAssignment(final String bundle) { return broker; } + private void policyFilter(final ServiceUnitId serviceUnit) { + // need multimap or at least set of RUs + 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 : brokerCandidateCache) { + final String brokerUrlString = String.format("http://%s", broker); + URL brokerUrl = null; + 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); + } + } + /** * As any broker, start the load manager. * 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 index b377285e98e32..562fe9acc823d 100644 --- 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 @@ -19,10 +19,6 @@ public class ModularLoadManagerWrapper implements LoadManager { private ModularLoadManager loadManager; - public ModularLoadManagerWrapper(final PulsarService pulsar) { - this(new ModularLoadManagerImpl(pulsar)); - } - public ModularLoadManagerWrapper(final ModularLoadManager loadManager) { this.loadManager = loadManager; } @@ -55,8 +51,7 @@ public String getBrokerRoot() { @Override public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { - return new SimpleResourceUnit( - String.format("http://%s", loadManager.selectBrokerForAssignment(serviceUnit.toString())), + return new SimpleResourceUnit(String.format("http://%s", loadManager.selectBrokerForAssignment(serviceUnit)), new PulsarResourceDescription()); } From 4f97664e9ba71052c6c3519473a1acc6f003c2c6 Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 23 Mar 2017 13:07:30 -0700 Subject: [PATCH 06/26] Make system resource usage more influential --- .../yahoo/pulsar/broker/LocalBrokerData.java | 30 +++++++++++++++++++ .../yahoo/pulsar/broker/PulsarService.java | 15 +++++----- .../impl/LeastLongTermMessageRate.java | 21 +++++++++++-- .../impl/ModularLoadManagerImpl.java | 4 +-- 4 files changed, 58 insertions(+), 12 deletions(-) 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 index bcb1aa1843187..09ecea04a4ea9 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -27,6 +27,9 @@ public class LocalBrokerData extends JSONWritable implements ServiceLookupData { 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; @@ -73,6 +76,8 @@ public LocalBrokerData(final String webServiceUrl, final String webServiceUrlTls cpu = new ResourceUsage(); memory = new ResourceUsage(); directMemory = new ResourceUsage(); + bandwidthIn = new ResourceUsage(); + bandwidthOut = new ResourceUsage(); bundles = new HashSet<>(); lastBundleGains = new HashSet<>(); lastBundleLosses = new HashSet<>(); @@ -101,6 +106,8 @@ private void updateSystemResourceUsage(final SystemResourceUsage systemResourceU 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 @@ -149,7 +156,13 @@ private void updateBundleData(final Map bundleStat 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() { @@ -176,6 +189,22 @@ 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; } @@ -299,4 +328,5 @@ public String getPulsarServiceUrl() { 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 dc48aa94c3a1f..4b65007b0864e 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 @@ -267,6 +267,14 @@ public void start() throws PulsarServerException { if (getZkClient().exists(DYNAMIC_LOAD_MANAGER_ZPATH, false) != null) { config.setLoadManagerClassName(new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, false, null))); } + + try { + ZkUtils.createFullPathOptimistic(getZkClient(), DYNAMIC_LOAD_MANAGER_ZPATH, + config.getLoadManagerClassName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + // Ignore + } + this.loadManager = new AtomicReference<>(LoadManager.create(this)); this.startLoadManagementService(); @@ -341,13 +349,6 @@ public synchronized void brokerIsAFollowerNow() { acquireSLANamespace(); - try { - ZkUtils.createFullPathOptimistic(getZkClient(), DYNAMIC_LOAD_MANAGER_ZPATH, - config.getLoadManagerClassName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException e) { - // Ignore - } - getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, new LoadManagerWatcher(), null); LOG.info("messaging service is ready, bootstrap service on port={}, broker url={}, cluster={}, configs={}", 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 index 186093e468fcc..8911d8b224aad 100644 --- 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 @@ -31,14 +31,29 @@ public LeastLongTermMessageRate(final ServiceConfiguration conf) { // Form a score for a broker using its preallocated bundle data and time // average data. - private static double getScore(final BrokerData brokerData) { + 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(); - return totalMessageRate + timeAverageData.getLongTermMsgRateIn() + timeAverageData.getLongTermMsgRateOut(); + 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; } /** @@ -62,7 +77,7 @@ public String selectBroker(final Set candidates, final BundleData bundle // 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)); + 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. 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 index ce8a4328611f7..74ddbbec89cec 100644 --- 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 @@ -198,7 +198,7 @@ private void updateAllBrokerData() { if (brokerDataMap.containsKey(broker)) { // Replace previous local broker data. - loadData.getBrokerData().get(broker).setLocalData(localData); + brokerDataMap.get(broker).setLocalData(localData); } else { // Initialize BrokerData object for previously unseen // brokers. @@ -485,7 +485,7 @@ private void policyFilter(final ServiceUnitId serviceUnit) { } for (final String broker : brokerCandidateCache) { final String brokerUrlString = String.format("http://%s", broker); - URL brokerUrl = null; + URL brokerUrl; try { brokerUrl = new URL(brokerUrlString); } catch (MalformedURLException e) { From b7ec53361b51562207e7af1ff71a4cdfa2d9575b Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 23 Mar 2017 14:18:34 -0700 Subject: [PATCH 07/26] Fix reflective setting of load manager in tests --- .../client/api/BrokerServiceLookupTest.java | 35 ++++++++++--------- 1 file changed, 18 insertions(+), 17 deletions(-) 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 4a059135b9d72..dbc34278be305 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; @@ -119,19 +120,19 @@ public void testMultipleBrokerLookup() throws Exception { pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); + AtomicReference loadManager1 = spy(pulsar.getLoadManager()); + AtomicReference loadManager2 = spy(pulsar2.getLoadManager()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: redirect request to leader [2] - doReturn(true).when(loadManager2).isCentralized(); + doReturn(true).when(loadManager2.get()).isCentralized(); loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); // mock: return Broker2 as a Least-loaded broker when leader receies request [3] - doReturn(true).when(loadManager1).isCentralized(); + doReturn(true).when(loadManager1.get()).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); + doReturn(resourceUnit).when(loadManager1.get()).getLeastLoaded(any(ServiceUnitId.class)); loadManagerField.set(pulsar.getNamespaceService(), loadManager1); /**** started broker-2 ****/ @@ -214,14 +215,14 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { stopBroker(); startBroker(); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); + AtomicReference loadManager2 = spy(pulsar2.getLoadManager()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: return Broker2 as a Least-loaded broker when leader receies request - doReturn(true).when(loadManager2).isCentralized(); + doReturn(true).when(loadManager2.get()).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); + doReturn(resourceUnit).when(loadManager2.get()).getLeastLoaded(any(ServiceUnitId.class)); loadManagerField.set(pulsar.getNamespaceService(), loadManager2); /**** started broker-2 ****/ @@ -290,17 +291,17 @@ public void testPartitionTopicLookup() throws Exception { pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); + AtomicReference loadManager1 = spy(pulsar.getLoadManager()); + AtomicReference loadManager2 = spy(pulsar2.getLoadManager()); 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(); + doReturn(true).when(loadManager1.get()).isCentralized(); loadManagerField.set(pulsar.getNamespaceService(), loadManager1); // mock: redirect request to leader - doReturn(true).when(loadManager2).isCentralized(); + doReturn(true).when(loadManager2.get()).isCentralized(); loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); /**** broker-2 started ****/ @@ -375,20 +376,20 @@ public void testWebserviceServiceTls() throws Exception { pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - LoadManager loadManager1 = spy(pulsar.getLoadManager().get()); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); + AtomicReference loadManager1 = spy(pulsar.getLoadManager()); + AtomicReference loadManager2 = spy(pulsar2.getLoadManager()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); // mock: redirect request to leader [2] - doReturn(true).when(loadManager2).isCentralized(); + doReturn(true).when(loadManager2.get()).isCentralized(); loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); // mock: return Broker2 as a Least-loaded broker when leader receies // request [3] - doReturn(true).when(loadManager1).isCentralized(); + doReturn(true).when(loadManager1.get()).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); + doReturn(resourceUnit).when(loadManager1.get()).getLeastLoaded(any(ServiceUnitId.class)); loadManagerField.set(pulsar.getNamespaceService(), loadManager1); /**** started broker-2 ****/ From 81b4725e0e643df68a6f6c091aeb463c1ecfadca Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 23 Mar 2017 14:57:17 -0700 Subject: [PATCH 08/26] Conform with mockito syntax --- .../client/api/BrokerServiceLookupTest.java | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) 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 dbc34278be305..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 @@ -120,20 +120,20 @@ public void testMultipleBrokerLookup() throws Exception { pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - AtomicReference loadManager1 = spy(pulsar.getLoadManager()); - AtomicReference 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.get()).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + doReturn(true).when(loadManager2).isCentralized(); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); // mock: return Broker2 as a Least-loaded broker when leader receies request [3] - doReturn(true).when(loadManager1.get()).isCentralized(); + doReturn(true).when(loadManager1).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager1.get()).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); /**** started broker-2 ****/ @@ -215,15 +215,15 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { stopBroker(); startBroker(); - AtomicReference loadManager2 = spy(pulsar2.getLoadManager()); + 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(loadManager2.get()).isCentralized(); + doReturn(true).when(loadManager2).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager2.get()).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager2); + doReturn(resourceUnit).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager2)); /**** started broker-2 ****/ // load namespace-bundle by calling Broker2 @@ -291,18 +291,18 @@ public void testPartitionTopicLookup() throws Exception { pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - AtomicReference loadManager1 = spy(pulsar.getLoadManager()); - AtomicReference 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.get()).isCentralized(); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + doReturn(true).when(loadManager1).isCentralized(); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); // mock: redirect request to leader - doReturn(true).when(loadManager2.get()).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + doReturn(true).when(loadManager2).isCentralized(); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); /**** broker-2 started ****/ ProducerConfiguration producerConf = new ProducerConfiguration(); @@ -376,21 +376,21 @@ public void testWebserviceServiceTls() throws Exception { pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); - AtomicReference loadManager1 = spy(pulsar.getLoadManager()); - AtomicReference 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.get()).isCentralized(); - loadManagerField.set(pulsar2.getNamespaceService(), loadManager2); + doReturn(true).when(loadManager2).isCentralized(); + loadManagerField.set(pulsar2.getNamespaceService(), new AtomicReference<>(loadManager2)); // mock: return Broker2 as a Least-loaded broker when leader receies // request [3] - doReturn(true).when(loadManager1.get()).isCentralized(); + doReturn(true).when(loadManager1).isCentralized(); SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar2.getWebServiceAddress(), null); - doReturn(resourceUnit).when(loadManager1.get()).getLeastLoaded(any(ServiceUnitId.class)); - loadManagerField.set(pulsar.getNamespaceService(), loadManager1); + doReturn(resourceUnit).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); + loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); /**** started broker-2 ****/ From 925ffcaa02d518b0618273fdd27f84cc11a64216 Mon Sep 17 00:00:00 2001 From: breese Date: Fri, 24 Mar 2017 14:08:20 -0700 Subject: [PATCH 09/26] Improve formatting and design --- .../com/yahoo/pulsar/broker/BrokerData.java | 88 +- .../com/yahoo/pulsar/broker/BundleData.java | 123 +- .../com/yahoo/pulsar/broker/JSONWritable.java | 40 +- .../yahoo/pulsar/broker/LocalBrokerData.java | 622 +++++---- .../pulsar/broker/TimeAverageBrokerData.java | 287 ++-- .../pulsar/broker/TimeAverageMessageData.java | 277 ++-- .../broker/loadbalance/BrokerFilter.java | 39 +- .../pulsar/broker/loadbalance/LoadData.java | 45 +- .../loadbalance/LoadSheddingStrategy.java | 24 +- .../loadbalance/ModularLoadManager.java | 119 +- .../ModularLoadManagerStrategy.java | 72 +- .../loadbalance/impl/DeviationShedder.java | 217 ++- .../impl/LeastLongTermMessageRate.java | 144 +- .../impl/ModularLoadManagerImpl.java | 1188 ++++++++-------- .../impl/ModularLoadManagerWrapper.java | 147 +- .../impl/SimpleLoadManagerImpl.java | 63 +- .../testclient/LoadSimulationController.java | 1218 ++++++++--------- .../testclient/LoadSimulationServer.java | 598 ++++---- .../ModularLoadManagerBrokerMonitor.java | 344 ++--- .../SimpleLoadManagerBrokerMonitor.java | 329 +++-- 20 files changed, 2956 insertions(+), 3028 deletions(-) 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 index e85a9f5624ae9..4bf798c718005 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java @@ -4,51 +4,49 @@ 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). + * 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; - } + 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 index c45d56acdb9f7..037f57554efc1 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java @@ -3,77 +3,76 @@ import com.yahoo.pulsar.common.policies.data.loadbalancer.NamespaceBundleStats; /** - * Data class comprising the short term and long term historical data for this - * bundle. + * 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; + // 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; + // 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() { - } + // 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 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); - } + /** + * 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); - } + /** + * 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 TimeAverageMessageData getShortTermData() { + return shortTermData; + } - public void setShortTermData(TimeAverageMessageData shortTermData) { - this.shortTermData = shortTermData; - } + public void setShortTermData(TimeAverageMessageData shortTermData) { + this.shortTermData = shortTermData; + } - public TimeAverageMessageData getLongTermData() { - return longTermData; - } + public TimeAverageMessageData getLongTermData() { + return longTermData; + } - public void setLongTermData(TimeAverageMessageData longTermData) { - this.longTermData = 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 index 1b23daa4232cc..1bfb8368186d9 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java @@ -9,25 +9,25 @@ */ 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 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); - } + /** + * 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 index 09ecea04a4ea9..1f23710fba5a3 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -16,317 +16,315 @@ */ 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; - } + // 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/TimeAverageBrokerData.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java index dec4a88532052..a2c279cceedcc 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java @@ -6,151 +6,148 @@ 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. + * 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; - } + 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 index 78fa16d5e2770..68fffd3622bbd 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java @@ -6,144 +6,141 @@ * 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; - } + // 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/loadbalance/BrokerFilter.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/BrokerFilter.java index 804f31d2e48dd..d8850b94eed81 100644 --- 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 @@ -6,28 +6,25 @@ 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. + * 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); + /** + * 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 index 069676e52af1e..113c68d4d0db8 100644 --- 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 @@ -7,33 +7,32 @@ import com.yahoo.pulsar.broker.BundleData; /** - * This class represents all data that could be relevant when making a load - * management decision. + * 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 broker names to their available data. + */ + private final Map brokerData; - /** - * Map from bundle names to their time-sensitive aggregated data. - */ - private final Map bundleData; + /** + * 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<>(); - } + /** + * Initialize a LoadData. + */ + public LoadData() { + this.brokerData = new ConcurrentHashMap<>(); + this.bundleData = new ConcurrentHashMap<>(); + } - public Map getBrokerData() { - return brokerData; - } + public Map getBrokerData() { + return brokerData; + } - public Map getBundleData() { - return bundleData; - } + public Map getBundleData() { + return bundleData; + } } 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 index 4e073ee6fb112..665757d76d24a 100644 --- 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 @@ -5,20 +5,18 @@ import com.yahoo.pulsar.broker.ServiceConfiguration; /** - * Load management component which determines the criteria for unloading - * bundles. + * 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 selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); + /** + * 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 selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); } 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 index 00cb3d7fc53d1..47ba489e33fec 100644 --- 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 @@ -4,79 +4,72 @@ 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. + * 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 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, 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(); + /** + * As the leader broker, attempt to automatically detect and split hot namespace bundles. + */ + void doNamespaceBundleSplit(); - /** - * Get the broker root ZooKeeper path. - */ - String getBrokerRoot(); + /** + * Get the broker root ZooKeeper path. + */ + String getBrokerRoot(); - /** - * 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 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, retrieve the namespace bundle stats and system resource - * usage to update data local to this broker. - */ - void updateLocalBrokerData(); + /** + * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. + */ + void updateLocalBrokerData(); - /** - * 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, 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, 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, write the local broker data to ZooKeeper. - */ - void writeBrokerDataOnZooKeeper(); + /** + * 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(); + /** + * 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 index af5a7a71044fa..3a2ce1f423dc8 100644 --- 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 @@ -7,45 +7,45 @@ import com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; /** - * Interface which serves as a component for ModularLoadManagerImpl, flexibly - * allowing the injection of potentially complex strategies. + * 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); + /** + * 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 { - final Class placementStrategyClass = Class.forName(conf.getModularPlacementStrategyClassName()); + /** + * 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 { + final Class placementStrategyClass = Class.forName(conf.getModularPlacementStrategyClassName()); - // Assume there is a constructor of one argument of - // ServiceConfiguration. - return (ModularLoadManagerStrategy) placementStrategyClass.getConstructor(ServiceConfiguration.class) - .newInstance(conf); - } catch (Exception e) { - // Ignore - } - return new LeastLongTermMessageRate(conf); - } + // Assume there is a constructor of one argument of + // ServiceConfiguration. + return (ModularLoadManagerStrategy) placementStrategyClass.getConstructor(ServiceConfiguration.class) + .newInstance(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 index df5692c45fb03..54a8d30e30073 100644 --- 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 @@ -13,123 +13,122 @@ import com.yahoo.pulsar.broker.loadbalance.LoadSheddingStrategy; /** - * An abstract class which makes a LoadSheddingStrategy which makes decisions - * based on standard deviation easier to implement. + * An abstract class which makes a LoadSheddingStrategy which makes decisions based on standard deviation easier to + * implement. */ 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; + // 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<>(); - } + /** + * 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 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); + // 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); + // 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 - * obsessive 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 selectBundlesForUnloading(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(); + /** + * 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 selectBundlesForUnloading(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(); + // 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(); + 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; - } + 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 index 8911d8b224aad..a9d66e9742682 100644 --- 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 @@ -16,80 +16,84 @@ import com.yahoo.pulsar.broker.loadbalance.ModularLoadManagerStrategy; /** - * Placement strategy which selects a broker based on which one has the least - * long term message rate. + * 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); + private static Logger log = LoggerFactory.getLogger(LeastLongTermMessageRate.class); - // Maintain this list to reduce object creation. - private ArrayList bestBrokers; + // Maintain this list to reduce object creation. + private ArrayList bestBrokers; - public LeastLongTermMessageRate(final ServiceConfiguration conf) { - bestBrokers = new ArrayList<>(); - } + public LeastLongTermMessageRate(final ServiceConfiguration conf) { + bestBrokers = new ArrayList<>(); + } - // Form a score for a broker using its preallocated bundle data and time - // average data. - 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; - } + // Form a score for a broker using its preallocated bundle data and time + // average data. + 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); - } - } - return bestBrokers.get(ThreadLocalRandom.current().nextInt(bestBrokers.size())); - } + /** + * 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/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 74ddbbec89cec..5f0cb1c905619 100644 --- 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 @@ -56,604 +56,594 @@ import com.yahoo.pulsar.zookeeper.ZooKeeperDataCache; public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCacheListener { - public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; - public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; - public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; - - private static final int MIBI = 1024 * 1024; - private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); - - private final LocalBrokerData localData; - private final LoadData loadData; - - // Used to determine whether a bundle is preallocated. - private final Map preallocatedBundleToBroker; - - // Set of broker candidates to reuse so that object creation is avoided. - private final Set brokerCandidateCache; - private final Set primariesCache; - private final Set sharedCache; - - // Used to filter brokers from being selected for assignment. - private final List filterPipeline; - - // Pipeline used to determine what namespaces, if any, should be unloaded. - private final List loadSheddingPipeline; - - // Strategy used to determine where new topics should be placed. - private final ModularLoadManagerStrategy placementStrategy; - - private final SimpleResourceAllocationPolicies policies; - - private final PulsarService pulsar; - private final ZooKeeper zkClient; - private final ServiceConfiguration conf; - private final BrokerHostUsage brokerHostUsage; - private final ZooKeeperDataCache brokerDataCache; - private final ZooKeeperChildrenCache availableActiveBrokers; - private final ScheduledExecutorService scheduler; - private final LoadingCache adminCache; - - // 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; - - // Timestamp of last invocation of updateBundleData. - private long lastBundleDataUpdate; - - private String brokerZnodePath; - private final String brokerRoot; - - // System resource usage directly after starting. - private SystemResourceUsage baselineSystemResourceUsage; - - public ModularLoadManagerImpl(final PulsarService pulsar) { - this(pulsar, LOADBALANCE_BROKERS_ROOT); - } - - /** - * Initialize this load manager. - * - * @param pulsar - * Client to construct this manager from. - * @param brokerRoot - * ZooKeeper path containing some data implementing - * ServiceLookup. - */ - public ModularLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { - this.brokerRoot = brokerRoot; - this.pulsar = pulsar; - zkClient = pulsar.getZkClient(); - conf = pulsar.getConfiguration(); - policies = new SimpleResourceAllocationPolicies(pulsar); - loadData = new LoadData(); - preallocatedBundleToBroker = new ConcurrentHashMap<>(); - brokerCandidateCache = new HashSet<>(); - primariesCache = new HashSet<>(); - sharedCache = new HashSet<>(); - filterPipeline = new ArrayList<>(); - loadSheddingPipeline = new ArrayList<>(); - localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); - placementStrategy = ModularLoadManagerStrategy.create(conf); - 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()); - } - }); - - // Initialize the default - defaultStats = new NamespaceBundleStats(); - defaultStats.msgThroughputIn = conf.getLoadManagerDefaultMessageThroughputIn(); - defaultStats.msgThroughputOut = conf.getLoadManagerDefaultMessageThroughputOut(); - defaultStats.msgRateIn = conf.getLoadManagerDefaultMessageRateIn(); - defaultStats.msgRateOut = conf.getLoadManagerDefaultMessageRateOut(); - if (SystemUtils.IS_OS_LINUX) { - brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); - } else { - brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); - } - - 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); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); - 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); - } - }); - scheduler = Executors.newScheduledThreadPool(1); - } - - /* - * 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", brokerRoot, 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); - } - } - - /* - * Use the Pulsar client to acquire the namespace bundle stats. - */ - private Map getBundleStats() { - return pulsar.getBrokerService().getBundleStats(); - } - - /** - * Update both the broker data and the bundle data. - */ - public void updateAll() { - updateAllBrokerData(); - updateBundleData(); - } - - /** - * 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. - */ - public 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); - } - } - - // 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() - + conf.getLoadManagerBrokerDataUpdateIntervalInSeconds() * 1000; - } - - // Determine if the bundle data requires an update by measuring the time - // past since the last update. - private boolean needBundleDataUpdate() { - return System.currentTimeMillis() > lastBundleDataUpdate - + conf.getLoadManagerBundleDataUpdateIntervalInSeconds() * 1000; - } - - // 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. - } - } - } - - // Get the ZooKeeper path for the given bundle full name. - public static String getBundleDataZooKeeperPath(final String bundle) { - return BUNDLE_DATA_ZPATH + "/" + bundle; - } - - // Get the total number of used bytes in the JVM. - private static long getRealtimeJVMHeapUsageBytes() { - return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); - } - - // Get the system resource usage for this broker. - 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(); - long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); - 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; - } - - // 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); - } - - // 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(conf.getLoadManagerNumberOfSamplesShortTermWindow(), - conf.getLoadManagerNumberOfSamplesLongTermWindow(), defaultStats); - } - return bundleData; - } - - /** - * 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); - } - } - } - - 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()); - } - - /** - * As the leader broker, select bundles for the namespace service to unload - * so that they may be reassigned to new brokers. - */ - @Override - public void doLoadShedding() { - for (LoadSheddingStrategy strategy : loadSheddingPipeline) { - final Map bundlesToUnload = strategy.selectBundlesForUnloading(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( - getNamespaceNameFromBundleName(bundle), 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? - } - - @Override - public String getBrokerRoot() { - return brokerRoot; - } - - /** - * 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(); - brokerCandidateCache.addAll(loadData.getBrokerData().keySet()); - policyFilter(serviceUnit); - - // 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; - } - - private void policyFilter(final ServiceUnitId serviceUnit) { - // need multimap or at least set of RUs - 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 : brokerCandidateCache) { - 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); - } - } - - /** - * 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, brokerRoot); - - String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); - brokerZnodePath = brokerRoot + "/" + 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(); - baselineSystemResourceUsage = getSystemResourceUsage(); - } 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 = getSystemResourceUsage(); - 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); - } - } - } - } + public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; + public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; + public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; + + private static final int MIBI = 1024 * 1024; + private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); + + private final LocalBrokerData localData; + private final LoadData loadData; + + // Used to determine whether a bundle is preallocated. + private final Map preallocatedBundleToBroker; + + // Set of broker candidates to reuse so that object creation is avoided. + private final Set brokerCandidateCache; + private final Set primariesCache; + private final Set sharedCache; + + // Used to filter brokers from being selected for assignment. + private final List filterPipeline; + + // Pipeline used to determine what namespaces, if any, should be unloaded. + private final List loadSheddingPipeline; + + // Strategy used to determine where new topics should be placed. + private final ModularLoadManagerStrategy placementStrategy; + + private final SimpleResourceAllocationPolicies policies; + + private final PulsarService pulsar; + private final ZooKeeper zkClient; + private final ServiceConfiguration conf; + private final BrokerHostUsage brokerHostUsage; + private final ZooKeeperDataCache brokerDataCache; + private final ZooKeeperChildrenCache availableActiveBrokers; + private final ScheduledExecutorService scheduler; + private final LoadingCache adminCache; + + // 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; + + // Timestamp of last invocation of updateBundleData. + private long lastBundleDataUpdate; + + private String brokerZnodePath; + private final String brokerRoot; + + // System resource usage directly after starting. + private SystemResourceUsage baselineSystemResourceUsage; + + public ModularLoadManagerImpl(final PulsarService pulsar) { + this(pulsar, LOADBALANCE_BROKERS_ROOT); + } + + /** + * Initialize this load manager. + * + * @param pulsar + * Client to construct this manager from. + * @param brokerRoot + * ZooKeeper path containing some data implementing ServiceLookup. + */ + public ModularLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { + this.brokerRoot = brokerRoot; + this.pulsar = pulsar; + zkClient = pulsar.getZkClient(); + conf = pulsar.getConfiguration(); + policies = new SimpleResourceAllocationPolicies(pulsar); + loadData = new LoadData(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + brokerCandidateCache = new HashSet<>(); + primariesCache = new HashSet<>(); + sharedCache = new HashSet<>(); + filterPipeline = new ArrayList<>(); + loadSheddingPipeline = new ArrayList<>(); + localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), + pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + placementStrategy = ModularLoadManagerStrategy.create(conf); + 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()); + } + }); + + // Initialize the default + defaultStats = new NamespaceBundleStats(); + defaultStats.msgThroughputIn = conf.getLoadManagerDefaultMessageThroughputIn(); + defaultStats.msgThroughputOut = conf.getLoadManagerDefaultMessageThroughputOut(); + defaultStats.msgRateIn = conf.getLoadManagerDefaultMessageRateIn(); + defaultStats.msgRateOut = conf.getLoadManagerDefaultMessageRateOut(); + if (SystemUtils.IS_OS_LINUX) { + brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); + } else { + brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); + } + + 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); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); + 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); + } + }); + scheduler = Executors.newScheduledThreadPool(1); + } + + /* + * 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", brokerRoot, 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); + } + } + + /* + * Use the Pulsar client to acquire the namespace bundle stats. + */ + private Map getBundleStats() { + return pulsar.getBrokerService().getBundleStats(); + } + + /** + * Update both the broker data and the bundle data. + */ + public void updateAll() { + updateAllBrokerData(); + updateBundleData(); + } + + /** + * 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. + */ + public 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); + } + } + + // 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() + + conf.getLoadManagerBrokerDataUpdateIntervalInSeconds() * 1000; + } + + // Determine if the bundle data requires an update by measuring the time + // past since the last update. + private boolean needBundleDataUpdate() { + return System.currentTimeMillis() > lastBundleDataUpdate + + conf.getLoadManagerBundleDataUpdateIntervalInSeconds() * 1000; + } + + // 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. + } + } + } + + // Get the ZooKeeper path for the given bundle full name. + public static String getBundleDataZooKeeperPath(final String bundle) { + return BUNDLE_DATA_ZPATH + "/" + bundle; + } + + // Get the total number of used bytes in the JVM. + private static long getRealtimeJVMHeapUsageBytes() { + return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + } + + // Get the system resource usage for this broker. + 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(); + long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); + 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; + } + + // 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); + } + + // 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(conf.getLoadManagerNumberOfSamplesShortTermWindow(), + conf.getLoadManagerNumberOfSamplesLongTermWindow(), defaultStats); + } + return bundleData; + } + + /** + * 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); + } + } + } + + 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()); + } + + /** + * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new + * brokers. + */ + @Override + public void doLoadShedding() { + for (LoadSheddingStrategy strategy : loadSheddingPipeline) { + final Map bundlesToUnload = strategy.selectBundlesForUnloading(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( + getNamespaceNameFromBundleName(bundle), 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? + } + + @Override + public String getBrokerRoot() { + return brokerRoot; + } + + /** + * 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(); + brokerCandidateCache.addAll(loadData.getBrokerData().keySet()); + policyFilter(serviceUnit); + + // 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; + } + + private void policyFilter(final ServiceUnitId serviceUnit) { + // need multimap or at least set of RUs + 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 : brokerCandidateCache) { + 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); + } + } + + /** + * 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, brokerRoot); + + String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); + brokerZnodePath = brokerRoot + "/" + 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(); + baselineSystemResourceUsage = getSystemResourceUsage(); + } 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 = getSystemResourceUsage(); + 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 index 562fe9acc823d..a0708db950d2c 100644 --- 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 @@ -13,80 +13,79 @@ import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; /** - * Wrapper class allowing classes of instance ModularLoadManager to be - * compatible with the interface LoadManager. + * 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 String getBrokerRoot() { - return loadManager.getBrokerRoot(); - } - - @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 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(); - } + 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 String getBrokerRoot() { + return loadManager.getBrokerRoot(); + } + + @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 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 0fa8864232ddf..17334176cb278 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 @@ -221,12 +221,12 @@ public PulsarAdmin load(String key) throws Exception { }); 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(), brokerRoot); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override @@ -253,8 +253,8 @@ public void start() throws PulsarServerException { ServiceConfiguration conf = pulsar.getConfiguration(); if (pulsar.getZkClient().exists(brokerRoot, false) == null) { try { - ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerRoot, new byte[0], - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerRoot, new byte[0], Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); } catch (KeeperException.NodeExistsException e) { // ignore the exception, node might be present already } @@ -387,13 +387,12 @@ private double getMemoryLoadFactorFromZK(double defaultValue) { @Override public boolean isCentralized() { String strategy = this.getLoadBalancerPlacementStrategy(); - if ( strategy.equals(LOADBALANCER_STRATEGY_LLS) || strategy.equals(LOADBALANCER_STRATEGY_LEAST_MSG) ) { + if (strategy.equals(LOADBALANCER_STRATEGY_LLS) || strategy.equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { return true; } return this.getDynamicConfigurationBoolean(LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH, - SETTING_NAME_IS_CENTRALIZED, - pulsar.getConfiguration().getLoadBalancerIsCentralized()); + SETTING_NAME_IS_CENTRALIZED, pulsar.getConfiguration().getLoadBalancerIsCentralized()); } private long getLoadBalancerBrokerUnderloadedThresholdPercentage() { @@ -420,7 +419,6 @@ private boolean getLoadBalancerAutoBundleSplitEnabled() { pulsar.getConfiguration().getLoadBalancerAutoBundleSplitEnabled()); } - /* * temp method, remove it in future, in-place to make this glue code to make load balancing work */ @@ -479,7 +477,7 @@ 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, @@ -762,7 +760,8 @@ private void updateLoadBalancingMetrics(String hostname, long finalRank, Resourc * preAllocatedQuota into calculation; 3) Everything (preAllocatedBundles and preAllocatedQuotas) will get reset in * load ranking. */ - private synchronized 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(); @@ -831,7 +830,8 @@ private synchronized ResourceUnit findBrokerForPlacement(Multimap getFinalCandidatesWithPolicy(NamespaceName namespace, - Multimap primaries, Multimap shared) { + Multimap primaries, Multimap shared) { Multimap finalCandidates = TreeMultimap.create(); // if not enough primary then it should be union of primaries and secondaries finalCandidates.putAll(primaries); @@ -891,7 +891,7 @@ private Multimap getFinalCandidatesNoPolicy(Multimap getFinalCandidates(ServiceUnitId serviceUnit, - Map> availableBrokers) { + Map> availableBrokers) { // need multimap or at least set of RUs Multimap matchedPrimaries = TreeMultimap.create(); Multimap matchedShared = TreeMultimap.create(); @@ -990,7 +990,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 @@ -1011,8 +1011,8 @@ private ResourceUnit getLeastLoadedBroker(ServiceUnitId serviceUnit, } if (finalCandidates.size() > 0) { - if (this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LLS) || - this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { + if (this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LLS) + || this.getLoadBalancerPlacementStrategy().equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { selectedBroker = findBrokerForPlacement(finalCandidates, serviceUnit); } else { selectedBroker = placementStrategy.findBrokerForPlacement(finalCandidates); @@ -1121,7 +1121,8 @@ public LoadReport generateLoadReport() throws Exception { try { LoadReport loadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); - loadReport.setName(String.format("%s:%s", pulsar.getAdvertisedAddress(), pulsar.getConfiguration().getWebServicePort())); + loadReport.setName(String.format("%s:%s", pulsar.getAdvertisedAddress(), + pulsar.getConfiguration().getWebServicePort())); SystemResourceUsage systemResourceUsage = this.getSystemResourceUsage(); loadReport.setOverLoaded( isAboveLoadLevel(systemResourceUsage, this.getLoadBalancerBrokerOverloadedThresholdPercentage())); @@ -1137,13 +1138,13 @@ public LoadReport generateLoadReport() throws Exception { final Set bundleGains = new HashSet<>(); final Set bundleLosses = new HashSet<>(); - for (String oldBundle: oldBundles) { + for (String oldBundle : oldBundles) { if (!newBundles.contains(oldBundle)) { bundleLosses.add(oldBundle); } } - for (String newBundle: newBundles) { + for (String newBundle : newBundles) { if (!oldBundles.contains(newBundle)) { bundleGains.add(newBundle); } @@ -1152,8 +1153,8 @@ public LoadReport generateLoadReport() throws Exception { loadReport.setBundleLosses(bundleLosses); final ResourceQuota allocatedQuota = getTotalAllocatedQuota(newBundles); - loadReport.setAllocatedCPU((allocatedQuota.getMsgRateIn() + allocatedQuota.getMsgRateOut()) - * realtimeCpuLoadFactor); + loadReport.setAllocatedCPU( + (allocatedQuota.getMsgRateIn() + allocatedQuota.getMsgRateOut()) * realtimeCpuLoadFactor); loadReport.setAllocatedMemory(allocatedQuota.getMemory()); loadReport.setAllocatedBandwidthIn(allocatedQuota.getBandwidthIn()); loadReport.setAllocatedBandwidthOut(allocatedQuota.getBandwidthOut()); @@ -1172,8 +1173,8 @@ public LoadReport generateLoadReport() throws Exception { final ResourceQuota preAllocatedQuota = getTotalAllocatedQuota(preAllocatedBundles); - loadReport.setPreAllocatedCPU((preAllocatedQuota.getMsgRateIn() + preAllocatedQuota.getMsgRateOut()) - * realtimeCpuLoadFactor); + loadReport.setPreAllocatedCPU( + (preAllocatedQuota.getMsgRateIn() + preAllocatedQuota.getMsgRateOut()) * realtimeCpuLoadFactor); loadReport.setPreAllocatedMemory(preAllocatedQuota.getMemory()); loadReport.setPreAllocatedBandwidthIn(preAllocatedQuota.getBandwidthIn()); loadReport.setPreAllocatedBandwidthOut(preAllocatedQuota.getBandwidthOut()); @@ -1226,7 +1227,7 @@ public void writeLoadReportOnZookeeper() throws Exception { lastLoadReport.getSystemResourceUsage(), pulsar.getLocalZkCacheService().getResourceQuotaCache().getDefaultQuota()); double bundlePercentageChange = (maxCapacity > 0) ? (bundleCountChange * 100 / maxCapacity) : 0; - if (newBundleCount != oldBundleCount ) { + if (newBundleCount != oldBundleCount) { needUpdate = true; } @@ -1244,15 +1245,15 @@ public void writeLoadReportOnZookeeper() throws Exception { ? ((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) + / newUsage.directMemory.limit) : 0; double bandwidthOutChange = (newUsage.bandwidthOut.limit > 0) ? ((newUsage.bandwidthOut.usage - oldUsage.bandwidthOut.usage) * 100 - / newUsage.bandwidthOut.limit) + / newUsage.bandwidthOut.limit) : 0; double bandwidthInChange = (newUsage.bandwidthIn.limit > 0) ? ((newUsage.bandwidthIn.usage - oldUsage.bandwidthIn.usage) * 100 - / newUsage.bandwidthIn.limit) + / newUsage.bandwidthIn.limit) : 0; long resourceChange = (long) Math.min(100.0, Math.max(Math.abs(cpuChange), 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 index b2b9fd41c4420..43c736ab0fdfe 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -31,619 +31,613 @@ import com.yahoo.pulsar.common.util.ObjectMapperFactory; /** - * 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. + * 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 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]; - System.out.format("Found %d servers:\n", 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()); - System.out.format("Connected to %s\n", 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) { - System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", 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) { - System.out.println("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); - System.out.println("Sending trade request to " + servers[i]); - outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); - writeProducerOptions(outputStreams[i], arguments, destination); - outputStreams[i].flush(); - if (inputStreams[i].read() != -1) { - System.out.println("Created producer and consumer for " + destination); - } else { - System.out.format("ERROR: Socket to %s closed\n", 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 { - System.out.println("Searching for server with topic " + destination); - for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); - foundTopic = true; - break; - case LoadSimulationServer.NO_SUCH_TOPIC: - break; - case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - break; - default: - System.out.println("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)) { - System.out.format("ERROR: Topic %s not found\n", 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)); - System.out.println("Searching for server with topic " + destination); - for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); - foundTopic = true; - break; - case LoadSimulationServer.NO_SUCH_TOPIC: - break; - case LoadSimulationServer.REDUNDANT_COMMAND: - System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); - foundTopic = true; - break; - case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - break; - default: - System.out.println("ERROR: Unknown response signal received: " + readValue); - } - } - if (!foundTopic) { - System.out.format("ERROR: Topic %s not found\n", 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(LoadSimulationServer.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) { - System.out.format("ERROR: Socket to %s closed\n", servers[i]); - } else if (foundOnServer == 0) { - System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, - servers[i]); - } else if (foundOnServer > 0) { - System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, tenant, - group, servers[i]); - numFound += foundOnServer; - } else { - System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, - servers[i]); - } - } - if (numFound == 0) { - System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); - } else { - System.out.format("Found %d topics belonging to tenant %s and group %s\n", 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(LoadSimulationServer.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); - System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); - ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - System.out.format("Creating new API data at %s\n", 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: - System.out.format("ERROR: Unknown command \"%s\"\n", 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(); - } + 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]; + System.out.format("Found %d servers:\n", 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()); + System.out.format("Connected to %s\n", 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) { + System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", 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) { + System.out.println("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); + System.out.println("Sending trade request to " + servers[i]); + outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); + writeProducerOptions(outputStreams[i], arguments, destination); + outputStreams[i].flush(); + if (inputStreams[i].read() != -1) { + System.out.println("Created producer and consumer for " + destination); + } else { + System.out.format("ERROR: Socket to %s closed\n", 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 { + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("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)) { + System.out.format("ERROR: Topic %s not found\n", 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)); + System.out.println("Searching for server with topic " + destination); + for (DataOutputStream outputStream : outputStreams) { + outputStream.write(LoadSimulationServer.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 LoadSimulationServer.FOUND_TOPIC: + System.out.format("Found topic %s on server %s\n", destination, servers[i]); + foundTopic = true; + break; + case LoadSimulationServer.NO_SUCH_TOPIC: + break; + case LoadSimulationServer.REDUNDANT_COMMAND: + System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); + foundTopic = true; + break; + case -1: + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + break; + default: + System.out.println("ERROR: Unknown response signal received: " + readValue); + } + } + if (!foundTopic) { + System.out.format("ERROR: Topic %s not found\n", 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(LoadSimulationServer.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) { + System.out.format("ERROR: Socket to %s closed\n", servers[i]); + } else if (foundOnServer == 0) { + System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, + servers[i]); + } else if (foundOnServer > 0) { + System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, tenant, + group, servers[i]); + numFound += foundOnServer; + } else { + System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, + servers[i]); + } + } + if (numFound == 0) { + System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); + } else { + System.out.format("Found %d topics belonging to tenant %s and group %s\n", 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(LoadSimulationServer.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); + System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); + ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + System.out.format("Creating new API data at %s\n", 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: + System.out.format("ERROR: Unknown command \"%s\"\n", 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/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index 09503acfe9470..afef70e5e5a70 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -36,322 +36,288 @@ import io.netty.util.concurrent.DefaultThreadFactory; /** - * LoadSimulationServer is used to simulate client load by maintaining producers - * and consumers for topics. Instances of this class are controlled across a - * network via LoadSimulationController. + * LoadSimulationServer 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 LoadSimulationServer { - // 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.createProducerAsync(topic, producerConf).get(); - } 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 LoadSimulationServer. - 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 LoadSimulationServer(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(); - - // Disable connection pooling. - clientConf.setConnectionsPerBroker(0); - - // 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 LoadSimulationServer(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. - System.out.println("Listening for controller command..."); - final Socket socket = serverSocket.accept(); - System.out.format("Connected to %s\n", socket.getInetAddress().getHostName()); - executor.submit(() -> { - try { - handle(socket); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - }); - } - } + // 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)); + } + + public void start() throws Exception { + Producer producer = producerFuture.get(); + final Consumer consumer = consumerFuture.get(); + while (!stop.get()) { + producer.sendAsync(payload.get()); + rateLimiter.acquire(); + } + producer.closeAsync(); + consumer.closeAsync(); + } + } + + // JCommander arguments for starting a LoadSimulationServer. + 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 LoadSimulationServer(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(); + + // Disable connection pooling. + clientConf.setConnectionsPerBroker(0); + + // 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 LoadSimulationServer(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. + System.out.println("Listening for controller command..."); + final Socket socket = serverSocket.accept(); + System.out.format("Connected to %s\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/ModularLoadManagerBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java index b1318f6fb0aed..fc6cd78986b0f 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java @@ -16,176 +16,176 @@ import com.yahoo.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; public class ModularLoadManagerBrokerMonitor { - private static final String BROKER_ROOT = "/loadbalance/new-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)) { - System.out.println("Lost broker: " + oldBroker); - } - } - for (String newBroker : newBrokers) { - if (!brokers.contains(newBroker)) { - System.out.println("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) { - System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); - System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); - System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); - System.out.format("Message Rate Out: %.2f msgs/s\n", 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); - } - - System.out.println("\nBroker Data for " + broker + ":"); - System.out.println("---------------"); - - System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); - System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); - System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); - System.out.println("Num Producers: " + localBrokerData.getNumProducers()); - - System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); - - System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); - - System.out - .println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); - - System.out.println("\nLatest Data:\n"); - 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); - } - System.out.println("\nShort Term Data:\n"); - printMessageData(timeAverageData.getShortTermMsgThroughputIn(), - timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), - timeAverageData.getShortTermMsgRateOut()); - - System.out.println("\nLong Term Data:\n"); - printMessageData(timeAverageData.getLongTermMsgThroughputIn(), - timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), - timeAverageData.getLongTermMsgRateOut()); - - System.out.println(); - if (!localBrokerData.getLastBundleGains().isEmpty()) { - for (String bundle : localBrokerData.getLastBundleGains()) { - System.out.println("Gained Bundle: " + bundle); - } - System.out.println(); - } - if (!localBrokerData.getLastBundleLosses().isEmpty()) { - for (String bundle : localBrokerData.getLastBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); - } - System.out.println(); - } - } - } - - 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); - } - } + private static final String BROKER_ROOT = "/loadbalance/new-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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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) { + System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); + System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); + System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); + System.out.format("Message Rate Out: %.2f msgs/s\n", 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); + } + + System.out.println("\nBroker Data for " + broker + ":"); + System.out.println("---------------"); + + System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); + System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); + System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); + System.out.println("Num Producers: " + localBrokerData.getNumProducers()); + + System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); + + System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); + + System.out + .println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); + + System.out.println("\nLatest Data:\n"); + 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); + } + System.out.println("\nShort Term Data:\n"); + printMessageData(timeAverageData.getShortTermMsgThroughputIn(), + timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), + timeAverageData.getShortTermMsgRateOut()); + + System.out.println("\nLong Term Data:\n"); + printMessageData(timeAverageData.getLongTermMsgThroughputIn(), + timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), + timeAverageData.getLongTermMsgRateOut()); + + System.out.println(); + if (!localBrokerData.getLastBundleGains().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!localBrokerData.getLastBundleLosses().isEmpty()) { + for (String bundle : localBrokerData.getLastBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + 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/SimpleLoadManagerBrokerMonitor.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java index 9108bd5530903..b28070a263330 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java @@ -12,171 +12,170 @@ 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. + * 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 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)) { - System.out.println("Lost broker: " + oldBroker); - } - } - for (String newBroker : newBrokers) { - if (!brokers.contains(newBroker)) { - System.out.println("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(); - - System.out.println("\nLoad Report for " + brokerName + ":"); - System.out.println("---------------"); - - System.out.println("\nNum Topics: " + loadReport.getNumTopics()); - System.out.println("Num Bundles: " + loadReport.getNumBundles()); - - System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); - System.out.println(String.format("Allocated CPU: %.2f%%", - percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); - System.out.println(String.format("Preallocated CPU: %.2f%%", - percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); - - System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); - System.out.println(String.format("Allocated Memory: %.2f%%", - percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); - System.out.println(String.format("Preallocated Memory: %.2f%%", - percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); - - System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); - System.out.println(String.format("Allocated Bandwidth In: %.2f%%", - percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); - System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", - percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); - - System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); - System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", - percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); - System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", - percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); - - System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); - - System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); - System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); - System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); - System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); - System.out.println(); - if (!loadReport.getBundleGains().isEmpty()) { - for (String bundle : loadReport.getBundleGains()) { - System.out.println("Gained Bundle: " + bundle); - } - System.out.println(); - } - if (!loadReport.getBundleLosses().isEmpty()) { - for (String bundle : loadReport.getBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); - } - System.out.println(); - } - } - } - - 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); - } - } + 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)) { + System.out.println("Lost broker: " + oldBroker); + } + } + for (String newBroker : newBrokers) { + if (!brokers.contains(newBroker)) { + System.out.println("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(); + + System.out.println("\nLoad Report for " + brokerName + ":"); + System.out.println("---------------"); + + System.out.println("\nNum Topics: " + loadReport.getNumTopics()); + System.out.println("Num Bundles: " + loadReport.getNumBundles()); + + System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); + System.out.println(String.format("Allocated CPU: %.2f%%", + percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); + System.out.println(String.format("Preallocated CPU: %.2f%%", + percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); + + System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); + System.out.println(String.format("Allocated Memory: %.2f%%", + percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); + System.out.println(String.format("Preallocated Memory: %.2f%%", + percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); + + System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); + System.out.println(String.format("Allocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); + + System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); + System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", + percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); + + System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); + + System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); + System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); + System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); + System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); + System.out.println(); + if (!loadReport.getBundleGains().isEmpty()) { + for (String bundle : loadReport.getBundleGains()) { + System.out.println("Gained Bundle: " + bundle); + } + System.out.println(); + } + if (!loadReport.getBundleLosses().isEmpty()) { + for (String bundle : loadReport.getBundleLosses()) { + System.out.println("Lost Bundle: " + bundle); + } + System.out.println(); + } + } + } + + 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); + } + } } From 8ae61c1cf6b1dbbec579aa14180c97f88117f142 Mon Sep 17 00:00:00 2001 From: breese Date: Fri, 24 Mar 2017 15:44:20 -0700 Subject: [PATCH 10/26] Revert to old simulation behavior, use reflective interface design --- .../yahoo/pulsar/broker/LocalBrokerData.java | 5 + .../yahoo/pulsar/broker/PulsarService.java | 4 +- .../broker/loadbalance/LoadManager.java | 183 ++++++++-------- .../loadbalance/ModularLoadManager.java | 16 +- .../impl/LinuxBrokerHostUsageImpl.java | 20 +- .../impl/ModularLoadManagerImpl.java | 85 ++++---- .../impl/ModularLoadManagerWrapper.java | 5 + .../impl/SimpleLoadManagerImpl.java | 57 ++--- .../data/loadbalancer/ServiceLookupData.java | 8 +- pulsar-testclient/pom.xml | 1 - .../testclient/LoadSimulationServer.java | 200 ++++++++++-------- 11 files changed, 326 insertions(+), 258 deletions(-) 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 index 1f23710fba5a3..74ea63e0d5347 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -84,7 +84,12 @@ public LocalBrokerData(final String webServiceUrl, final String webServiceUrlTls /** * Using the system resource usage and bundle stats acquired from the Pulsar client, update this LocalBrokerData. + * <<<<<<< Updated upstream * + * ======= + * + * >>>>>>> Stashed changes + * * @param systemResourceUsage * System resource usage (cpu, memory, and direct memory). * @param bundleStats 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 4b65007b0864e..63cac69a3e2eb 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 @@ -210,7 +210,7 @@ public void close() throws PulsarServerException { private class LoadManagerWatcher implements Watcher { public void process(final WatchedEvent event) { - new Thread(() -> { + executor.execute(() -> { try { LOG.info("Attempting to change load manager"); final String newLoadManagerName = @@ -226,7 +226,7 @@ public void process(final WatchedEvent event) { } catch (Exception ex) { LOG.warn("Failed to change load manager due to {}", ex); } - }).start(); + }); } } 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 3a910d4edf381..027cfa04427f2 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 @@ -27,96 +27,103 @@ import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport; /** - * LoadManager runs though set of load reports collected from different brokers - * and generates a recommendation of namespace/ServiceUnit placement on - * machines/ResourceUnit. Each Concrete Load Manager will use different - * algorithms to generate this mapping. + * LoadManager runs though set of load reports collected from different brokers and generates a recommendation of + * namespace/ServiceUnit placement on machines/ResourceUnit. Each Concrete Load Manager will use different algorithms to + * generate this mapping. * - * Concrete Load Manager is also return the least loaded broker that should own - * the new namespace. + * Concrete Load Manager is also return the least loaded broker that should own the new namespace. */ public interface LoadManager { - public void start() throws PulsarServerException; - - /** - * Is centralized decision making to assign a new bundle. - */ - boolean isCentralized(); - - /** - * Returns the Least Loaded Resource Unit decided by some algorithm or - * criteria which is implementation specific - */ - ResourceUnit getLeastLoaded(ServiceUnitId su) throws Exception; - - /** - * Generate the load report - */ - LoadReport generateLoadReport() throws Exception; - - /** - * Set flag to force load report update - */ - void setLoadReportForceUpdateFlag(); - - /** - * Publish the current load report on ZK - */ - void writeLoadReportOnZookeeper() throws Exception; - - /** - * Update namespace bundle resource quota on ZK - */ - void writeResourceQuotasToZooKeeper() throws Exception; - - /** - * Generate load balancing stats metrics - */ - List getLoadBalancingMetrics(); - - /** - * Unload a candidate service unit to balance the load - */ - void doLoadShedding(); - - /** - * Namespace bundle split - */ - void doNamespaceBundleSplit() throws Exception; - - /** - * Determine the broker root. - */ - String getBrokerRoot(); - - /** - * Removes visibility of current broker from loadbalancer list so, other - * brokers can't redirect any request to this broker and this broker won't - * accept new connection requests. - * - * @throws Exception - */ - public void disableBroker() throws Exception; - - public void stop() throws PulsarServerException; - - 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.getConstructor(PulsarService.class).newInstance(pulsar); - if (loadManagerInstance instanceof LoadManager) { - return (LoadManager) loadManagerInstance; - } else if (loadManagerInstance instanceof ModularLoadManager) { - return new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance); - } - } catch (Exception e) { - // Ignore - } - // If we failed to create a load manager, default to - // SimpleLoadManagerImpl. - return new SimpleLoadManagerImpl(pulsar); - } + public void start() throws PulsarServerException; + + /** + * Is centralized decision making to assign a new bundle. + */ + boolean isCentralized(); + + /** + * Returns the Least Loaded Resource Unit decided by some algorithm or criteria which is implementation specific + */ + ResourceUnit getLeastLoaded(ServiceUnitId su) throws Exception; + + /** + * Generate the load report + */ + LoadReport generateLoadReport() throws Exception; + + /** + * Set flag to force load report update + */ + void setLoadReportForceUpdateFlag(); + + /** + * Publish the current load report on ZK + */ + void writeLoadReportOnZookeeper() throws Exception; + + /** + * Update namespace bundle resource quota on ZK + */ + void writeResourceQuotasToZooKeeper() throws Exception; + + /** + * Generate load balancing stats metrics + */ + List getLoadBalancingMetrics(); + + /** + * Unload a candidate service unit to balance the load + */ + void doLoadShedding(); + + /** + * Namespace bundle split + */ + void doNamespaceBundleSplit() throws Exception; + + /** + * Determine the broker root. + */ + String getBrokerRoot(); + + /** + * Removes visibility of current broker from loadbalancer list so, other brokers can't redirect any request to this + * broker and this broker won't accept new connection requests. + * + * @throws Exception + */ + 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) { + // Ignore + } + // 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/ModularLoadManager.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/ModularLoadManager.java index 47ba489e33fec..7ca960044dc31 100644 --- 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 @@ -1,6 +1,7 @@ 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; /** @@ -33,6 +34,11 @@ public interface ModularLoadManager { */ String getBrokerRoot(); + /** + * 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. * @@ -42,11 +48,6 @@ public interface ModularLoadManager { */ String selectBrokerForAssignment(ServiceUnitId serviceUnit); - /** - * As any broker, retrieve the namespace bundle stats and system resource usage to update data local to this broker. - */ - void updateLocalBrokerData(); - /** * As any broker, start the load manager. * @@ -63,6 +64,11 @@ public interface ModularLoadManager { */ 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. */ 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 d7571a1021caf..bbcd215d4fd5d 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 @@ -160,17 +160,17 @@ private List getNics() { } private boolean isPhysicalNic(Path path) { - path = Files.isSymbolicLink(path) ? path.toAbsolutePath() : 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; - } + path = Files.isSymbolicLink(path) ? path.toAbsolutePath() : 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; } - return false; + } + return false; } private Path getNicSpeedPath(String nic) { 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 index 5f0cb1c905619..c55b3dfc23e25 100644 --- 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 @@ -63,7 +63,7 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach private static final int MIBI = 1024 * 1024; private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); - private final LocalBrokerData localData; + private LocalBrokerData localData; private final LoadData loadData; // Used to determine whether a bundle is preallocated. @@ -81,18 +81,18 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach private final List loadSheddingPipeline; // Strategy used to determine where new topics should be placed. - private final ModularLoadManagerStrategy placementStrategy; + private ModularLoadManagerStrategy placementStrategy; - private final SimpleResourceAllocationPolicies policies; + private SimpleResourceAllocationPolicies policies; - private final PulsarService pulsar; - private final ZooKeeper zkClient; - private final ServiceConfiguration conf; - private final BrokerHostUsage brokerHostUsage; - private final ZooKeeperDataCache brokerDataCache; - private final ZooKeeperChildrenCache availableActiveBrokers; + private PulsarService pulsar; + private ZooKeeper zkClient; + private ServiceConfiguration conf; + private BrokerHostUsage brokerHostUsage; + private ZooKeeperDataCache brokerDataCache; + private ZooKeeperChildrenCache availableActiveBrokers; private final ScheduledExecutorService scheduler; - private final LoadingCache adminCache; + private LoadingCache adminCache; // The default bundle stats which are used to initialize historic data. // This data is overriden after the bundle receives its first sample. @@ -102,36 +102,39 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach private long lastBundleDataUpdate; private String brokerZnodePath; - private final String brokerRoot; // System resource usage directly after starting. private SystemResourceUsage baselineSystemResourceUsage; - public ModularLoadManagerImpl(final PulsarService pulsar) { - this(pulsar, LOADBALANCE_BROKERS_ROOT); + // Initialize fields when they do not depend on PulsarService. + public ModularLoadManagerImpl() { + loadData = new LoadData(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + brokerCandidateCache = new HashSet<>(); + primariesCache = new HashSet<>(); + sharedCache = new HashSet<>(); + filterPipeline = new ArrayList<>(); + loadSheddingPipeline = new ArrayList<>(); + defaultStats = new NamespaceBundleStats(); + scheduler = Executors.newScheduledThreadPool(1); } /** - * Initialize this load manager. + * Initialize this load manager using the given PulsarService. * * @param pulsar - * Client to construct this manager from. - * @param brokerRoot - * ZooKeeper path containing some data implementing ServiceLookup. + * The service to initialize with. */ - public ModularLoadManagerImpl(final PulsarService pulsar, final String brokerRoot) { - this.brokerRoot = brokerRoot; + public void initialize(final PulsarService pulsar) { + if (SystemUtils.IS_OS_LINUX) { + brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); + } else { + brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); + } this.pulsar = pulsar; zkClient = pulsar.getZkClient(); conf = pulsar.getConfiguration(); policies = new SimpleResourceAllocationPolicies(pulsar); - loadData = new LoadData(); - preallocatedBundleToBroker = new ConcurrentHashMap<>(); - brokerCandidateCache = new HashSet<>(); - primariesCache = new HashSet<>(); - sharedCache = new HashSet<>(); - filterPipeline = new ArrayList<>(); - loadSheddingPipeline = new ArrayList<>(); localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); placementStrategy = ModularLoadManagerStrategy.create(conf); @@ -148,17 +151,11 @@ public PulsarAdmin load(String key) throws Exception { } }); - // Initialize the default - defaultStats = new NamespaceBundleStats(); + // Initialize the default stats to assume for unseen bundles. defaultStats.msgThroughputIn = conf.getLoadManagerDefaultMessageThroughputIn(); defaultStats.msgThroughputOut = conf.getLoadManagerDefaultMessageThroughputOut(); defaultStats.msgRateIn = conf.getLoadManagerDefaultMessageRateIn(); defaultStats.msgRateOut = conf.getLoadManagerDefaultMessageRateOut(); - if (SystemUtils.IS_OS_LINUX) { - brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); - } else { - brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); - } brokerDataCache = new ZooKeeperDataCache(pulsar.getLocalZkCache()) { @Override @@ -167,7 +164,7 @@ public LocalBrokerData deserialize(String key, byte[] content) throws Exception } }; brokerDataCache.registerListener(this); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LOADBALANCE_BROKERS_ROOT); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override public void onUpdate(String path, Set data, Stat stat) { @@ -177,7 +174,17 @@ public void onUpdate(String path, Set data, Stat stat) { scheduler.submit(ModularLoadManagerImpl.this::updateAll); } }); - scheduler = Executors.newScheduledThreadPool(1); + } + + /** + * Initialize this load manager. + * + * @param pulsar + * Client to construct this manager from. + */ + public ModularLoadManagerImpl(final PulsarService pulsar) { + this(); + initialize(pulsar); } /* @@ -190,7 +197,7 @@ private void updateAllBrokerData() { final Map brokerDataMap = loadData.getBrokerData(); for (String broker : activeBrokers) { try { - String key = String.format("%s/%s", brokerRoot, broker); + String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker); final LocalBrokerData localData = brokerDataCache.get(key) .orElseThrow(KeeperException.NoNodeException::new); @@ -421,7 +428,7 @@ public void doNamespaceBundleSplit() { @Override public String getBrokerRoot() { - return brokerRoot; + return LOADBALANCE_BROKERS_ROOT; } /** @@ -548,10 +555,10 @@ private void policyFilter(final ServiceUnitId serviceUnit) { public void start() throws PulsarServerException { try { // Register the brokers in zk list - createZPathIfNotExists(zkClient, brokerRoot); + createZPathIfNotExists(zkClient, LOADBALANCE_BROKERS_ROOT); String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); - brokerZnodePath = brokerRoot + "/" + lookupServiceAddress; + brokerZnodePath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + lookupServiceAddress; updateLocalBrokerData(); try { 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 index a0708db950d2c..7cdb9b4e7741d 100644 --- 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 @@ -59,6 +59,11 @@ public List getLoadBalancingMetrics() { return Collections.emptyList(); } + @Override + public void initialize(final PulsarService pulsar) { + loadManager.initialize(pulsar); + } + @Override public boolean isCentralized() { return true; 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 17334176cb278..554503664e086 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 @@ -128,11 +128,11 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene 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"; @@ -159,9 +159,8 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene public static final String LOADBALANCER_STRATEGY_LEAST_MSG = "leastMsgPerSecond"; private String brokerZnodePath; - private final String brokerRoot; 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 @@ -173,13 +172,9 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene // flag to force update load report private boolean forceLoadReportUpdate = false; - public SimpleLoadManagerImpl(PulsarService pulsar) { - this(pulsar, LOADBALANCE_BROKERS_ROOT); - } - - public SimpleLoadManagerImpl(PulsarService pulsar, final String brokerRoot) { - this.brokerRoot = brokerRoot; - 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<>(); @@ -187,13 +182,18 @@ public SimpleLoadManagerImpl(PulsarService pulsar, final String brokerRoot) { this.realtimeResourceQuotas.set(new HashMap<>()); this.realtimeAvgResourceQuota = new ResourceQuota(); placementStrategy = new WRRPlacementStrategy(); - lastLoadReport = new LoadReport(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); + } + + @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 { @@ -227,7 +227,8 @@ public Long load(String key) throws Exception { return System.currentTimeMillis(); } }); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), brokerRoot); + + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LOADBALANCE_BROKERS_ROOT); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override public void onUpdate(String path, Set data, Stat stat) { @@ -237,13 +238,17 @@ 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 String getBrokerRoot() { - return brokerRoot; + return LOADBALANCE_BROKERS_ROOT; } @Override @@ -251,16 +256,16 @@ public void start() throws PulsarServerException { try { // Register the brokers in zk list ServiceConfiguration conf = pulsar.getConfiguration(); - if (pulsar.getZkClient().exists(brokerRoot, false) == null) { + if (pulsar.getZkClient().exists(LOADBALANCE_BROKERS_ROOT, false) == null) { try { - ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerRoot, new byte[0], Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); + ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), LOADBALANCE_BROKERS_ROOT, new byte[0], + 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 = brokerRoot + "/" + lookupServiceAddress; + brokerZnodePath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; LoadReport loadReport = null; try { loadReport = generateLoadReport(); @@ -973,7 +978,7 @@ private Map> getAvailableBrokers(ServiceUnitId serviceUn if (availableBrokers.isEmpty()) { // Create a map with all available brokers with no load information - Set activeBrokers = availableActiveBrokers.get(brokerRoot); + Set activeBrokers = availableActiveBrokers.get(LOADBALANCE_BROKERS_ROOT); List brokersToShuffle = new ArrayList<>(activeBrokers); Collections.shuffle(brokersToShuffle); activeBrokers = new HashSet<>(brokersToShuffle); @@ -1046,7 +1051,7 @@ private void updateRanking() { Set activeBrokers = availableActiveBrokers.get(); for (String broker : activeBrokers) { try { - String key = String.format("%s/%s", brokerRoot, broker); + String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker); LoadReport lr = loadReportCacheZk.get(key) .orElseThrow(() -> new KeeperException.NoNodeException()); ResourceUnit ru = new SimpleResourceUnit(String.format("http://%s", lr.getName()), 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 index 45cca44392e02..31e6b7d029e53 100644 --- 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 @@ -2,11 +2,11 @@ // For backwards compatibility purposes. public interface ServiceLookupData { - public String getWebServiceUrl(); + public String getWebServiceUrl(); - public String getWebServiceUrlTls(); + public String getWebServiceUrlTls(); - public String getPulsarServiceUrl(); + public String getPulsarServiceUrl(); - public String getPulsarServiceUrlTls(); + public String getPulsarServiceUrlTls(); } diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 4bb16cc6eadc8..c6b03f4575600 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -41,7 +41,6 @@ org.apache.zookeeper zookeeper - 3.4.8 ${project.groupId} diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index afef70e5e5a70..a0c40e0f93fd1 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -36,8 +36,9 @@ import io.netty.util.concurrent.DefaultThreadFactory; /** - * LoadSimulationServer is used to simulate client load by maintaining producers and consumers for topics. Instances of - * this class are controlled across a network via LoadSimulationController. + * LoadSimulationServer 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 LoadSimulationServer { // Values for command responses. @@ -81,8 +82,8 @@ private static class TradeUnit { final Map payloadCache; public TradeUnit(final TradeConfiguration tradeConf, final PulsarClient client, - final ProducerConfiguration producerConf, final ConsumerConfiguration consumerConf, - final Map payloadCache) throws Exception { + 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<>(); @@ -104,15 +105,48 @@ public void change(final TradeConfiguration tradeConf) { 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.createProducerAsync(topic, producerConf).get(); + } 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()) { - producer.sendAsync(payload.get()); - rateLimiter.acquire(); + 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(); + } } - producer.closeAsync(); - consumer.closeAsync(); } } @@ -175,81 +209,81 @@ private void handle(final byte command, final DataInputStream inputStream, final 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); + 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); } - }); - // 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); + 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); } - } - 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; + 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(numStopped); - break; - default: - throw new IllegalArgumentException("Unrecognized command code received: " + command); + 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(); } @@ -261,9 +295,9 @@ public LoadSimulationServer(final MainArguments arguments) throws Exception { topicsToTradeUnits = new ConcurrentHashMap<>(); final EventLoopGroup eventLoopGroup = SystemUtils.IS_OS_LINUX ? new EpollEventLoopGroup(Runtime.getRuntime().availableProcessors(), - new DefaultThreadFactory("pulsar-test-client")) + new DefaultThreadFactory("pulsar-test-client")) : new NioEventLoopGroup(Runtime.getRuntime().availableProcessors(), - new DefaultThreadFactory("pulsar-test-client")); + new DefaultThreadFactory("pulsar-test-client")); clientConf = new ClientConfiguration(); // Disable connection pooling. @@ -320,4 +354,4 @@ public void run() throws Exception { }); } } -} +} \ No newline at end of file From 9b5358b615198e897ef2ac0a12b5dda7a80b5e7e Mon Sep 17 00:00:00 2001 From: breese Date: Fri, 24 Mar 2017 15:45:40 -0700 Subject: [PATCH 11/26] Remove unnecessary async call --- .../java/com/yahoo/pulsar/testclient/LoadSimulationServer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index a0c40e0f93fd1..ebfa3af4d6a50 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -111,7 +111,7 @@ public void change(final TradeConfiguration tradeConf) { private Producer getNewProducer() throws Exception { while (true) { try { - return client.createProducerAsync(topic, producerConf).get(); + return client.createProducer(topic, producerConf); } catch (Exception e) { Thread.sleep(10000); } From 653a8537a7d4ef226c7ec3e910a2f341cac0edf4 Mon Sep 17 00:00:00 2001 From: breese Date: Fri, 24 Mar 2017 15:47:22 -0700 Subject: [PATCH 12/26] Fix formatting --- .../testclient/LoadSimulationServer.java | 157 +++++++++--------- 1 file changed, 78 insertions(+), 79 deletions(-) diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index ebfa3af4d6a50..e063ecefe4b5b 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -36,9 +36,8 @@ import io.netty.util.concurrent.DefaultThreadFactory; /** - * LoadSimulationServer is used to simulate client load by maintaining producers - * and consumers for topics. Instances of this class are controlled across a - * network via LoadSimulationController. + * LoadSimulationServer 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 LoadSimulationServer { // Values for command responses. @@ -82,8 +81,8 @@ private static class TradeUnit { final Map payloadCache; public TradeUnit(final TradeConfiguration tradeConf, final PulsarClient client, - final ProducerConfiguration producerConf, final ConsumerConfiguration consumerConf, - final Map payloadCache) throws Exception { + 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<>(); @@ -209,81 +208,81 @@ private void handle(final byte command, final DataInputStream inputStream, final 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. + 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 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); - } + } + 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); } - 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; - } + }); + // 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(numStopped); - break; - default: - throw new IllegalArgumentException("Unrecognized command code received: " + command); + } + 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(); } @@ -295,9 +294,9 @@ public LoadSimulationServer(final MainArguments arguments) throws Exception { topicsToTradeUnits = new ConcurrentHashMap<>(); final EventLoopGroup eventLoopGroup = SystemUtils.IS_OS_LINUX ? new EpollEventLoopGroup(Runtime.getRuntime().availableProcessors(), - new DefaultThreadFactory("pulsar-test-client")) + new DefaultThreadFactory("pulsar-test-client")) : new NioEventLoopGroup(Runtime.getRuntime().availableProcessors(), - new DefaultThreadFactory("pulsar-test-client")); + new DefaultThreadFactory("pulsar-test-client")); clientConf = new ClientConfiguration(); // Disable connection pooling. From 220376874d9b5c3cdf934a2812e45384cd4f78c1 Mon Sep 17 00:00:00 2001 From: breese Date: Fri, 24 Mar 2017 15:52:52 -0700 Subject: [PATCH 13/26] Remove git merge remnant --- .../main/java/com/yahoo/pulsar/broker/LocalBrokerData.java | 5 ----- .../com/yahoo/pulsar/testclient/LoadSimulationServer.java | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) 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 index 74ea63e0d5347..0d4f5e2b6f2bf 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -84,11 +84,6 @@ public LocalBrokerData(final String webServiceUrl, final String webServiceUrlTls /** * Using the system resource usage and bundle stats acquired from the Pulsar client, update this LocalBrokerData. - * <<<<<<< Updated upstream - * - * ======= - * - * >>>>>>> Stashed changes * * @param systemResourceUsage * System resource usage (cpu, memory, and direct memory). diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java index e063ecefe4b5b..c91bcbe15fef9 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java @@ -353,4 +353,4 @@ public void run() throws Exception { }); } } -} \ No newline at end of file +} From 0d4e74d2db0591b0f045f2632e24a7f39d3aec6c Mon Sep 17 00:00:00 2001 From: breese Date: Mon, 27 Mar 2017 17:01:48 -0700 Subject: [PATCH 14/26] Refactor some classes and put loadManagerClassName in dynamic conf --- bin/pulsar-perf | 6 +- conf/broker.conf | 27 -- .../pulsar/broker/ServiceConfiguration.java | 101 +---- .../com/yahoo/pulsar/broker/BrokerData.java | 15 + .../com/yahoo/pulsar/broker/BundleData.java | 15 + .../com/yahoo/pulsar/broker/JSONWritable.java | 15 + .../yahoo/pulsar/broker/LocalBrokerData.java | 15 + .../yahoo/pulsar/broker/PulsarService.java | 36 -- .../pulsar/broker/TimeAverageBrokerData.java | 15 + .../pulsar/broker/TimeAverageMessageData.java | 15 + .../pulsar/broker/admin/BrokerStats.java | 2 +- .../yahoo/pulsar/broker/admin/Brokers.java | 3 +- .../broker/loadbalance/BrokerFilter.java | 15 + .../pulsar/broker/loadbalance/LoadData.java | 15 + .../broker/loadbalance/LoadManager.java | 12 +- .../loadbalance/LoadSheddingStrategy.java | 17 +- .../loadbalance/ModularLoadManager.java | 20 +- .../ModularLoadManagerStrategy.java | 23 +- .../loadbalance/impl/DeviationShedder.java | 21 +- .../impl/LeastLongTermMessageRate.java | 23 +- .../impl/LinuxBrokerHostUsageImpl.java | 46 +-- .../impl/ModularLoadManagerImpl.java | 59 +-- .../impl/ModularLoadManagerWrapper.java | 20 +- .../impl/SimpleLoadManagerImpl.java | 146 +++---- .../broker/namespace/NamespaceService.java | 4 +- .../pulsar/broker/service/BrokerService.java | 13 + .../data/loadbalancer/ServiceLookupData.java | 15 + .../testclient/LoadSimulationClient.java | 371 ++++++++++++++++++ .../testclient/LoadSimulationController.java | 35 +- .../ModularLoadManagerBrokerMonitor.java | 17 +- .../SimpleLoadManagerBrokerMonitor.java | 15 + 31 files changed, 823 insertions(+), 329 deletions(-) create mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java diff --git a/bin/pulsar-perf b/bin/pulsar-perf index dd39891d67e0b..83b9715850874 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -77,7 +77,7 @@ where command is one of: consume Run a consumer simple-monitor Continuously receive broker data when using SimpleLoadManagerImpl modular-monitor Continuously receive broker data when using ModularLoadManagerImpl - simulation-server Run a simulation server acting as a Pulsar client + 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 @@ -146,8 +146,8 @@ elif [ "$COMMAND" == "simple-monitor" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.SimpleLoadManagerBrokerMonitor "$@" elif [ "$COMMAND" == "modular-monitor" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.ModularLoadManagerBrokerMonitor "$@" -elif [ "$COMMAND" == "simulation-server" ]; then - exec $JAVA $OPTS com.yahoo.pulsar.testclient.LoadSimulationServer "$@" +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 diff --git a/conf/broker.conf b/conf/broker.conf index d7f467c77c7a1..9737beb449417 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -297,32 +297,5 @@ keepAliveIntervalSeconds=30 # How often broker checks for inactive topics to be deleted (topics with no subscriptions and no one connected) brokerServicePurgeInactiveFrequencyInSeconds=60 -# Number of samples to use for short term time window -loadManagerNumberOfSamplesShortTermWindow=50 - -# Number of samples to use for long term time window -loadManagerNumberOfSamplesLongTermWindow=1000 - -# How often in seconds to update the broker data -loadManagerBrokerDataUpdateIntervalInSeconds=60 - -# How often in seconds to update the bundle data -loadManagerBundleDataUpdateIntervalInSeconds=60 - -# Default throughput in to assume for new bundles -loadManagerDefaultMessageThroughputIn=50000 - -# Default throughput out to assume for new bundles -loadManagerDefaultMessageThroughputOut=50000 - -# Default message rate in to assume for new bundles -loadManagerDefaultMessageRateIn=50 - -# Default message rate out to assume for new bundles -loadManagerDefaultMessageRateOut=50 - # Name of load manager to use loadManagerClassName=com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl - -# Name of placement strategy to use for new load manager API -loadManagerPlacementStrategyClassName=com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate 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 6cd3d3b9cd49e..2fd7dc6c287de 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 @@ -976,101 +976,10 @@ public void setReplicatorPrefix(String replicatorPrefix) { // Configurations for new load manager API - // Number of samples to use for short term time window - private int loadManagerNumberOfSamplesShortTermWindow = 50; - - // Number of samples to use for long term time window - private int loadManagerNumberOfSamplesLongTermWindow = 1000; - - // How often in seconds to update the broker data - private long loadManagerBrokerDataUpdateIntervalInSeconds = 60; - - // How often in seconds to update the bundle data - private long loadManagerBundleDataUpdateIntervalInSeconds = 60; - - // Default throughput in to assume for new bundles - private double loadManagerDefaultMessageThroughputIn = 50000; - - // Default throughput out to assume for new bundles - private double loadManagerDefaultMessageThroughputOut = 50000; - - // Default message rate in to assume for new bundles - private double loadManagerDefaultMessageRateIn = 50; - - // Default message rate out to assume for new bundles - private double loadManagerDefaultMessageRateOut = 50; - // Name of load manager to use + @FieldContext(dynamic = true) private String loadManagerClassName = "com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl"; - // Name of placement strategy to use for new loadbalancer API. - private String modularPlacementStrategyClassName = - "com.yahoo.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate"; - - public int getLoadManagerNumberOfSamplesShortTermWindow() { - return loadManagerNumberOfSamplesShortTermWindow; - } - - public void setLoadManagerNumberOfSamplesShortTermWindow(int loadManagerNumberOfSamplesShortTermWindow) { - this.loadManagerNumberOfSamplesShortTermWindow = loadManagerNumberOfSamplesShortTermWindow; - } - - public int getLoadManagerNumberOfSamplesLongTermWindow() { - return loadManagerNumberOfSamplesLongTermWindow; - } - - public void setLoadManagerNumberOfSamplesLongTermWindow(int loadManagerNumberOfSamplesLongTermWindow) { - this.loadManagerNumberOfSamplesLongTermWindow = loadManagerNumberOfSamplesLongTermWindow; - } - - public long getLoadManagerBrokerDataUpdateIntervalInSeconds() { - return loadManagerBrokerDataUpdateIntervalInSeconds; - } - - public void setLoadManagerBrokerDataUpdateIntervalInSeconds(long loadManagerBrokerDataUpdateIntervalInSeconds) { - this.loadManagerBrokerDataUpdateIntervalInSeconds = loadManagerBrokerDataUpdateIntervalInSeconds; - } - - public long getLoadManagerBundleDataUpdateIntervalInSeconds() { - return loadManagerBundleDataUpdateIntervalInSeconds; - } - - public void setLoadManagerBundleDataUpdateIntervalInSeconds(long loadManagerBundleDataUpdateIntervalInSeconds) { - this.loadManagerBundleDataUpdateIntervalInSeconds = loadManagerBundleDataUpdateIntervalInSeconds; - } - - public double getLoadManagerDefaultMessageThroughputIn() { - return loadManagerDefaultMessageThroughputIn; - } - - public void setLoadManagerDefaultMessageThroughputIn(double loadManagerDefaultMessageThroughputIn) { - this.loadManagerDefaultMessageThroughputIn = loadManagerDefaultMessageThroughputIn; - } - - public double getLoadManagerDefaultMessageThroughputOut() { - return loadManagerDefaultMessageThroughputOut; - } - - public void setLoadManagerDefaultMessageThroughputOut(double loadManagerDefaultMessageThroughputOut) { - this.loadManagerDefaultMessageThroughputOut = loadManagerDefaultMessageThroughputOut; - } - - public double getLoadManagerDefaultMessageRateIn() { - return loadManagerDefaultMessageRateIn; - } - - public void setLoadManagerDefaultMessageRateIn(double loadManagerDefaultMessageRateIn) { - this.loadManagerDefaultMessageRateIn = loadManagerDefaultMessageRateIn; - } - - public double getLoadManagerDefaultMessageRateOut() { - return loadManagerDefaultMessageRateOut; - } - - public void setLoadManagerDefaultMessageRateOut(double loadManagerDefaultMessageRateOut) { - this.loadManagerDefaultMessageRateOut = loadManagerDefaultMessageRateOut; - } - public String getLoadManagerClassName() { return loadManagerClassName; } @@ -1078,12 +987,4 @@ public String getLoadManagerClassName() { public void setLoadManagerClassName(String loadManagerClassName) { this.loadManagerClassName = loadManagerClassName; } - - public String getModularPlacementStrategyClassName() { - return modularPlacementStrategyClassName; - } - - public void setModularPlacementStrategyClassName(String modularPlacementStrategyClassName) { - this.modularPlacementStrategyClassName = modularPlacementStrategyClassName; - } } 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 index 4bf798c718005..33f081334b3fb 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BrokerData.java @@ -1,3 +1,18 @@ +/** + * 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; 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 index 037f57554efc1..e07cd017e529e 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/BundleData.java @@ -1,3 +1,18 @@ +/** + * 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; 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 index 1bfb8368186d9..3de8a29474f9a 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/JSONWritable.java @@ -1,3 +1,18 @@ +/** + * 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; 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 index 0d4f5e2b6f2bf..2249b2dfa4bf8 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/LocalBrokerData.java @@ -1,3 +1,18 @@ +/** + * 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; 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 63cac69a3e2eb..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 @@ -80,7 +80,6 @@ */ public class PulsarService implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(PulsarService.class); - private static final String DYNAMIC_LOAD_MANAGER_ZPATH = "/loadbalance/settings/load-manager"; private ServiceConfiguration config = null; private NamespaceService nsservice = null; private ManagedLedgerClientFactory managedLedgerClientFactory = null; @@ -208,28 +207,6 @@ public void close() throws PulsarServerException { } } - private class LoadManagerWatcher implements Watcher { - public void process(final WatchedEvent event) { - executor.execute(() -> { - try { - LOG.info("Attempting to change load manager"); - final String newLoadManagerName = - new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, this, null)); - - config.setLoadManagerClassName(newLoadManagerName); - final LoadManager newLoadManager = LoadManager.create(PulsarService.this); - LOG.info("Created load manager: {}", newLoadManagerName); - loadManager.get().disableBroker(); - newLoadManager.start(); - loadManager.set(newLoadManager); - - } catch (Exception ex) { - LOG.warn("Failed to change load manager due to {}", ex); - } - }); - } - } - /** * Get the current service configuration. * @@ -264,17 +241,6 @@ public void start() throws PulsarServerException { this.brokerService = new BrokerService(this); // Start load management service (even if load balancing is disabled) - if (getZkClient().exists(DYNAMIC_LOAD_MANAGER_ZPATH, false) != null) { - config.setLoadManagerClassName(new String(getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, false, null))); - } - - try { - ZkUtils.createFullPathOptimistic(getZkClient(), DYNAMIC_LOAD_MANAGER_ZPATH, - config.getLoadManagerClassName().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException e) { - // Ignore - } - this.loadManager = new AtomicReference<>(LoadManager.create(this)); this.startLoadManagementService(); @@ -349,8 +315,6 @@ public synchronized void brokerIsAFollowerNow() { acquireSLANamespace(); - getZkClient().getData(DYNAMIC_LOAD_MANAGER_ZPATH, new LoadManagerWatcher(), null); - LOG.info("messaging service is ready, bootstrap service on port={}, broker url={}, cluster={}, configs={}", config.getWebServicePort(), brokerServiceUrl, config.getClusterName(), config); } catch (Exception e) { 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 index a2c279cceedcc..43beb90b8f768 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageBrokerData.java @@ -1,3 +1,18 @@ +/** + * 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; 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 index 68fffd3622bbd..22cd0fcaab52b 100644 --- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/TimeAverageMessageData.java @@ -1,3 +1,18 @@ +/** + * 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; 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 21e3961669b18..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 @@ -175,7 +175,7 @@ public Map> getBrokerResourceAvailability(@PathPa if (lm instanceof SimpleLoadManagerImpl) { return ((SimpleLoadManagerImpl) lm).getResourceAvailabilityFor(ns).asMap(); } else { - return Collections.emptyMap(); + return null; } } catch (Exception e) { log.error("Unable to get Resource Availability - [{}]", 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 cffca7fffb64b..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(pulsar().getLoadManager().get().getBrokerRoot()); + 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 index d8850b94eed81..b2fc77450945f 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; 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 index 113c68d4d0db8..f78dcd98b09e2 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; 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 027cfa04427f2..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 @@ -25,6 +25,8 @@ 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 @@ -34,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; @@ -82,11 +87,6 @@ public interface LoadManager { */ void doNamespaceBundleSplit() throws Exception; - /** - * Determine the broker root. - */ - String getBrokerRoot(); - /** * Removes visibility of current broker from loadbalancer list so, other brokers can't redirect any request to this * broker and this broker won't accept new connection requests. @@ -121,7 +121,7 @@ static LoadManager create(final PulsarService pulsar) { return casted; } } catch (Exception e) { - // Ignore + 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/LoadSheddingStrategy.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/LoadSheddingStrategy.java index 665757d76d24a..f9835087b7a6c 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -18,5 +33,5 @@ public interface LoadSheddingStrategy { * The service configuration. * @return A map from all selected bundles to the brokers on which they reside. */ - Map selectBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); + Map findBundlesForUnloading(LoadData loadData, ServiceConfiguration conf); } 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 index 7ca960044dc31..14268d5ec9fc0 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -29,11 +44,6 @@ public interface ModularLoadManager { */ void doNamespaceBundleSplit(); - /** - * Get the broker root ZooKeeper path. - */ - String getBrokerRoot(); - /** * Initialize this load manager using the given pulsar service. */ 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 index 3a2ce1f423dc8..694e2593fa809 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -37,12 +52,8 @@ String selectBroker(Set candidates, BundleData bundleToAssign, LoadData */ static ModularLoadManagerStrategy create(final ServiceConfiguration conf) { try { - final Class placementStrategyClass = Class.forName(conf.getModularPlacementStrategyClassName()); - - // Assume there is a constructor of one argument of - // ServiceConfiguration. - return (ModularLoadManagerStrategy) placementStrategyClass.getConstructor(ServiceConfiguration.class) - .newInstance(conf); + // Only one strategy at the moment. + return new LeastLongTermMessageRate(conf); } catch (Exception e) { // Ignore } 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 index 54a8d30e30073..7906b7040fe0c 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -14,7 +29,9 @@ /** * An abstract class which makes a LoadSheddingStrategy which makes decisions based on standard deviation easier to - * implement. + * 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. @@ -50,7 +67,7 @@ public DeviationShedder() { * @return A map from all selected bundles to the brokers on which they reside. */ @Override - public Map selectBundlesForUnloading(final LoadData loadData, final ServiceConfiguration conf) { + public Map findBundlesForUnloading(final LoadData loadData, final ServiceConfiguration conf) { final Map result = new HashMap<>(); bundleTreeSetCache.clear(); metricTreeSetCache.clear(); 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 index a9d66e9742682..8766c148a0406 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -28,8 +43,12 @@ public LeastLongTermMessageRate(final ServiceConfiguration conf) { bestBrokers = new ArrayList<>(); } - // Form a score for a broker using its preallocated bundle data and time - // average data. + // 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; 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 bbcd215d4fd5d..6092b516c15c9 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 @@ -112,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]); @@ -149,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); @@ -160,15 +154,21 @@ private List getNics() { } private boolean isPhysicalNic(Path path) { - path = Files.isSymbolicLink(path) ? path.toAbsolutePath() : 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; + try { + path = path.toAbsolutePath(); + 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; + } } + } catch (IOException e) { + LOG.error("Failed to read link target for NIC " + path, e); + return false; } return false; } 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 index c55b3dfc23e25..2358806b59e15 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -16,6 +31,7 @@ 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; @@ -56,7 +72,6 @@ import com.yahoo.pulsar.zookeeper.ZooKeeperDataCache; public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCacheListener { - public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/new-brokers"; public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; public static final String BUNDLE_DATA_ZPATH = "/loadbalance/bundle-data"; @@ -103,8 +118,9 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach private String brokerZnodePath; - // System resource usage directly after starting. - private SystemResourceUsage baselineSystemResourceUsage; + // Hard-coded number of samples for short-term and long-term time windows. + private final int numLongSamples = 1000; + private final int numShortSamples = 10; // Initialize fields when they do not depend on PulsarService. public ModularLoadManagerImpl() { @@ -151,11 +167,11 @@ public PulsarAdmin load(String key) throws Exception { } }); - // Initialize the default stats to assume for unseen bundles. - defaultStats.msgThroughputIn = conf.getLoadManagerDefaultMessageThroughputIn(); - defaultStats.msgThroughputOut = conf.getLoadManagerDefaultMessageThroughputOut(); - defaultStats.msgRateIn = conf.getLoadManagerDefaultMessageRateIn(); - defaultStats.msgRateOut = conf.getLoadManagerDefaultMessageRateOut(); + // Initialize the default stats to assume for unseen bundles (hard-coded for now). + defaultStats.msgThroughputIn = 50000; + defaultStats.msgThroughputOut = 50000; + defaultStats.msgRateIn = 50; + defaultStats.msgRateOut = 50; brokerDataCache = new ZooKeeperDataCache(pulsar.getLocalZkCache()) { @Override @@ -164,7 +180,8 @@ public LocalBrokerData deserialize(String key, byte[] content) throws Exception } }; brokerDataCache.registerListener(this); - availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LOADBALANCE_BROKERS_ROOT); + availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), + LoadManager.LOADBALANCE_BROKERS_ROOT); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @Override public void onUpdate(String path, Set data, Stat stat) { @@ -197,7 +214,7 @@ private void updateAllBrokerData() { final Map brokerDataMap = loadData.getBrokerData(); for (String broker : activeBrokers) { try { - String key = String.format("%s/%s", LOADBALANCE_BROKERS_ROOT, broker); + String key = String.format("%s/%s", LoadManager.LOADBALANCE_BROKERS_ROOT, broker); final LocalBrokerData localData = brokerDataCache.get(key) .orElseThrow(KeeperException.NoNodeException::new); @@ -288,14 +305,14 @@ public void updateBundleData() { // past since the last update. private boolean needBrokerDataUpdate() { return System.currentTimeMillis() > localData.getLastUpdate() - + conf.getLoadManagerBrokerDataUpdateIntervalInSeconds() * 1000; + + 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 - + conf.getLoadManagerBundleDataUpdateIntervalInSeconds() * 1000; + + TimeUnit.MINUTES.toMillis(conf.getLoadBalancerResourceQuotaUpdateIntervalMinutes()); } // Attempt to create a ZooKeeper path if it does not exist. @@ -357,8 +374,7 @@ private BundleData getBundleDataOrDefault(final String bundle) { log.warn("Error when trying to find bundle {} on zookeeper: {}", bundle, e); } if (bundleData == null) { - bundleData = new BundleData(conf.getLoadManagerNumberOfSamplesShortTermWindow(), - conf.getLoadManagerNumberOfSamplesLongTermWindow(), defaultStats); + bundleData = new BundleData(numShortSamples, numLongSamples, defaultStats); } return bundleData; } @@ -399,9 +415,9 @@ private String getBundleRangeFromBundleName(String bundleName) { * brokers. */ @Override - public void doLoadShedding() { + public synchronized void doLoadShedding() { for (LoadSheddingStrategy strategy : loadSheddingPipeline) { - final Map bundlesToUnload = strategy.selectBundlesForUnloading(loadData, conf); + final Map bundlesToUnload = strategy.findBundlesForUnloading(loadData, conf); if (bundlesToUnload != null && !bundlesToUnload.isEmpty()) { try { for (Map.Entry entry : bundlesToUnload.entrySet()) { @@ -426,11 +442,6 @@ public void doNamespaceBundleSplit() { // TODO? } - @Override - public String getBrokerRoot() { - return LOADBALANCE_BROKERS_ROOT; - } - /** * When the broker data ZooKeeper nodes are updated, update the broker data map. */ @@ -476,7 +487,6 @@ public synchronized String selectBrokerForAssignment(final ServiceUnitId service } private void policyFilter(final ServiceUnitId serviceUnit) { - // need multimap or at least set of RUs primariesCache.clear(); sharedCache.clear(); NamespaceName namespace = serviceUnit.getNamespaceObject(); @@ -555,10 +565,10 @@ private void policyFilter(final ServiceUnitId serviceUnit) { public void start() throws PulsarServerException { try { // Register the brokers in zk list - createZPathIfNotExists(zkClient, LOADBALANCE_BROKERS_ROOT); + createZPathIfNotExists(zkClient, LoadManager.LOADBALANCE_BROKERS_ROOT); String lookupServiceAddress = pulsar.getAdvertisedAddress() + ":" + conf.getWebServicePort(); - brokerZnodePath = LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; + brokerZnodePath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + lookupServiceAddress; final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + lookupServiceAddress; updateLocalBrokerData(); try { @@ -573,7 +583,6 @@ public void start() throws PulsarServerException { zkClient.setData(timeAverageZPath, (new TimeAverageBrokerData()).getJsonBytes(), -1); updateAll(); lastBundleDataUpdate = System.currentTimeMillis(); - baselineSystemResourceUsage = getSystemResourceUsage(); } catch (Exception e) { log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e); throw new PulsarServerException(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 index 7cdb9b4e7741d..498083a8320a3 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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; @@ -43,11 +58,6 @@ public LoadReport generateLoadReport() { return null; } - @Override - public String getBrokerRoot() { - return loadManager.getBrokerRoot(); - } - @Override public ResourceUnit getLeastLoaded(final ServiceUnitId serviceUnit) { return new SimpleResourceUnit(String.format("http://%s", loadManager.selectBrokerForAssignment(serviceUnit)), 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 554503664e086..839862760b3b9 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 @@ -98,6 +98,10 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene // load balancing metrics private AtomicReference> loadBalancingMetrics = new AtomicReference<>(); + // 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) @@ -134,9 +138,7 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene 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"; - public static final String LOADBALANCER_DYNAMIC_SETTING_SECONDARY_STRATEGY_ZPATH = "/loadbalance/settings/secondary_strategy"; public static final String LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH = "/loadbalance/settings/is_centralized"; 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"; @@ -147,7 +149,6 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private static final String SETTING_NAME_LOAD_FACTOR_CPU = "loadFactorCPU"; private static final String SETTING_NAME_LOAD_FACTOR_MEM = "loadFactorMemory"; private static final String SETTING_NAME_STRATEGY = "loadBalancerStrategy"; - private static final String SETTING_NAME_SECONDARY_STRATEGY = "loadBalancerSecondaryStrategy"; private static final String SETTING_NAME_IS_CENTRALIZED = "loadBalancerIsCentralized"; private static final String SETTING_NAME_OVERLOAD_THRESHOLD = "overloadThreshold"; private static final String SETTING_NAME_UNDERLOAD_THRESHOLD = "underloadThreshold"; @@ -182,6 +183,8 @@ public SimpleLoadManagerImpl() { this.realtimeResourceQuotas.set(new HashMap<>()); this.realtimeAvgResourceQuota = new ResourceQuota(); placementStrategy = new WRRPlacementStrategy(); + bundleGainsCache = new HashSet<>(); + bundleLossesCache = new HashSet<>(); } @Override @@ -246,11 +249,6 @@ public SimpleLoadManagerImpl(PulsarService pulsar) { initialize(pulsar); } - @Override - public String getBrokerRoot() { - return LOADBALANCE_BROKERS_ROOT; - } - @Override public void start() throws PulsarServerException { try { @@ -1119,76 +1117,78 @@ private SystemResourceUsage getSystemResourceUsage() throws IOException { @Override public LoadReport generateLoadReport() throws Exception { - 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(); - final Set bundleGains = new HashSet<>(); - final Set bundleLosses = new HashSet<>(); - - for (String oldBundle : oldBundles) { - if (!newBundles.contains(oldBundle)) { - bundleLosses.add(oldBundle); - } + 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); + } + } - for (String newBundle : newBundles) { - if (!oldBundles.contains(newBundle)) { - bundleGains.add(newBundle); + 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<>(); } - } - loadReport.setBundleGains(bundleGains); - loadReport.setBundleLosses(bundleLosses); - - 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); + 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; + 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; + } } } 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 f65f544c1ddd3..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 @@ -405,7 +405,7 @@ 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", loadManager.get().getBrokerRoot(), uri.getHost(), + String path = String.format("%s/%s:%s", LoadManager.LOADBALANCE_BROKERS_ROOT, uri.getHost(), uri.getPort()); pulsar.getLocalZkCache().getDataAsync(path, serviceLookupDataDeserializer).thenAccept(reportData -> { if (reportData.isPresent()) { @@ -427,7 +427,7 @@ private CompletableFuture createLookupResult(String candidateBroke } private boolean isBrokerActive(String candidateBroker) throws KeeperException, InterruptedException { - Set activeNativeBrokers = pulsar.getLocalZkCache().getChildren(loadManager.get().getBrokerRoot()); + Set activeNativeBrokers = pulsar.getLocalZkCache().getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT); for (String brokerHostPort : activeNativeBrokers) { if (candidateBroker.equals("http://" + brokerHostPort)) { 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 943afad955c87..1c56d7baa8b24 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; @@ -910,6 +911,18 @@ private void updateConfigurationAndRegisterListeners() { // add listener on "maxConcurrentTopicLoadRequest" value change registerConfigurationListener("maxConcurrentTopicLoadRequest", (maxConcurrentTopicLoadRequest) -> topicLoadRequestSemaphore.set(new Semaphore((int) maxConcurrentTopicLoadRequest, false))); + registerConfigurationListener("loadManagerClassName", className -> { + try { + log.info("Attempting to change load manager"); + 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-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 index 31e6b7d029e53..5940f53ec1fec 100644 --- 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 @@ -1,3 +1,18 @@ +/** + * 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. 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..25eede82a1e03 --- /dev/null +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java @@ -0,0 +1,371 @@ +/** + * 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; + +/** + * 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 { + // 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(); + + // Disable connection pooling. + clientConf.setConnectionsPerBroker(0); + + // 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. + System.out.println("Listening for controller command..."); + final Socket socket = serverSocket.accept(); + System.out.format("Connected to %s\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 index 43c736ab0fdfe..a907c32167eb0 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -1,3 +1,18 @@ +/** + * 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; @@ -259,7 +274,7 @@ private synchronized void trade(final ShellArguments arguments, final String des // the controller. final int i = random.nextInt(servers.length); System.out.println("Sending trade request to " + servers[i]); - outputStreams[i].write(LoadSimulationServer.TRADE_COMMAND); + outputStreams[i].write(LoadSimulationClient.TRADE_COMMAND); writeProducerOptions(outputStreams[i], arguments, destination); outputStreams[i].flush(); if (inputStreams[i].read() != -1) { @@ -286,7 +301,7 @@ private void handleTrade(final ShellArguments arguments) throws Exception { private synchronized boolean change(final ShellArguments arguments, final String destination) throws Exception { System.out.println("Searching for server with topic " + destination); for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.CHANGE_COMMAND); + outputStream.write(LoadSimulationClient.CHANGE_COMMAND); writeProducerOptions(outputStream, arguments, destination); outputStream.flush(); } @@ -294,11 +309,11 @@ private synchronized boolean change(final ShellArguments arguments, final String for (int i = 0; i < servers.length; ++i) { int readValue; switch (readValue = inputStreams[i].read()) { - case LoadSimulationServer.FOUND_TOPIC: + case LoadSimulationClient.FOUND_TOPIC: System.out.format("Found topic %s on server %s\n", destination, servers[i]); foundTopic = true; break; - case LoadSimulationServer.NO_SUCH_TOPIC: + case LoadSimulationClient.NO_SUCH_TOPIC: break; case -1: System.out.format("ERROR: Socket to %s closed\n", servers[i]); @@ -332,7 +347,7 @@ private void handleStop(final ShellArguments arguments) throws Exception { commandArguments.get(3)); System.out.println("Searching for server with topic " + destination); for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.STOP_COMMAND); + outputStream.write(LoadSimulationClient.STOP_COMMAND); outputStream.writeUTF(destination); outputStream.flush(); } @@ -340,13 +355,13 @@ private void handleStop(final ShellArguments arguments) throws Exception { for (int i = 0; i < servers.length; ++i) { int readValue; switch (readValue = inputStreams[i].read()) { - case LoadSimulationServer.FOUND_TOPIC: + case LoadSimulationClient.FOUND_TOPIC: System.out.format("Found topic %s on server %s\n", destination, servers[i]); foundTopic = true; break; - case LoadSimulationServer.NO_SUCH_TOPIC: + case LoadSimulationClient.NO_SUCH_TOPIC: break; - case LoadSimulationServer.REDUNDANT_COMMAND: + case LoadSimulationClient.REDUNDANT_COMMAND: System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); foundTopic = true; break; @@ -394,7 +409,7 @@ private void handleGroupChange(final ShellArguments arguments) throws Exception final String tenant = commandArguments.get(1); final String group = commandArguments.get(2); for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.CHANGE_GROUP_COMMAND); + outputStream.write(LoadSimulationClient.CHANGE_GROUP_COMMAND); outputStream.writeUTF(tenant); outputStream.writeUTF(group); outputStream.writeInt(arguments.size); @@ -440,7 +455,7 @@ private void handleGroupStop(final ShellArguments arguments) throws Exception { final String tenant = commandArguments.get(1); final String group = commandArguments.get(2); for (DataOutputStream outputStream : outputStreams) { - outputStream.write(LoadSimulationServer.STOP_GROUP_COMMAND); + outputStream.write(LoadSimulationClient.STOP_GROUP_COMMAND); outputStream.writeUTF(tenant); outputStream.writeUTF(group); outputStream.flush(); 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 index fc6cd78986b0f..ce083c2b22695 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java @@ -1,3 +1,18 @@ +/** + * 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; @@ -16,7 +31,7 @@ import com.yahoo.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; public class ModularLoadManagerBrokerMonitor { - private static final String BROKER_ROOT = "/loadbalance/new-brokers"; + 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(); 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 index b28070a263330..39827e3b6b746 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java @@ -1,3 +1,18 @@ +/** + * 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; From aa2d580f9affb8142064d17d5329400750c4a075 Mon Sep 17 00:00:00 2001 From: breese Date: Tue, 28 Mar 2017 12:23:20 -0700 Subject: [PATCH 15/26] Delete LoadSimulationServer --- .../testclient/LoadSimulationServer.java | 356 ------------------ 1 file changed, 356 deletions(-) delete mode 100644 pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java diff --git a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java deleted file mode 100644 index c91bcbe15fef9..0000000000000 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationServer.java +++ /dev/null @@ -1,356 +0,0 @@ -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; - -/** - * LoadSimulationServer 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 LoadSimulationServer { - // 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 LoadSimulationServer. - 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 LoadSimulationServer(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(); - - // Disable connection pooling. - clientConf.setConnectionsPerBroker(0); - - // 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 LoadSimulationServer(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. - System.out.println("Listening for controller command..."); - final Socket socket = serverSocket.accept(); - System.out.format("Connected to %s\n", socket.getInetAddress().getHostName()); - executor.submit(() -> { - try { - handle(socket); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - }); - } - } -} From 358d1ff4b312a634605d1965dcd71105599455f5 Mon Sep 17 00:00:00 2001 From: breese Date: Tue, 28 Mar 2017 16:40:09 -0700 Subject: [PATCH 16/26] Clean API, share more code --- .../loadbalance/impl/LoadManagerShared.java | 157 +++++++ .../impl/ModularLoadManagerImpl.java | 424 ++++++++---------- .../impl/SimpleLoadManagerImpl.java | 148 ++---- 3 files changed, 368 insertions(+), 361 deletions(-) create mode 100644 pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LoadManagerShared.java 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..816db7c2794e0 --- /dev/null +++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -0,0 +1,157 @@ +/** + * 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) { + 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 : brokerCandidateCache) { + 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 index 2358806b59e15..415ac705e07d9 100644 --- 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 @@ -62,7 +62,6 @@ 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.NamespaceName; 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; @@ -72,88 +71,118 @@ import com.yahoo.pulsar.zookeeper.ZooKeeperDataCache; public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCacheListener { - public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; + 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"; - private static final int MIBI = 1024 * 1024; - private static final Logger log = LoggerFactory.getLogger(ModularLoadManagerImpl.class); + // Default message rate to assume for unseen bundles. + public static final double DEFAULT_MESSAGE_RATE = 50; - private LocalBrokerData localData; - private final LoadData loadData; + // 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; - // Used to determine whether a bundle is preallocated. - private final Map preallocatedBundleToBroker; + // Value of prefix "mibi" (e.g., number of bytes in a mibibyte). + public static final int MIBI = 1024 * 1024; + + // 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; - private final Set primariesCache; - private final Set sharedCache; + + // 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; - private ZooKeeper zkClient; - private ServiceConfiguration conf; - private BrokerHostUsage brokerHostUsage; - private ZooKeeperDataCache brokerDataCache; - private ZooKeeperChildrenCache availableActiveBrokers; - private final ScheduledExecutorService scheduler; - private LoadingCache adminCache; - // 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; + // Cache for primary brokers according to policies. + private final Set primariesCache; - // Timestamp of last invocation of updateBundleData. - private long lastBundleDataUpdate; + // Executor service used to regularly update broker data. + private final ScheduledExecutorService scheduler; - private String brokerZnodePath; + // Cache for shard brokers according to policies. + private final Set sharedCache; - // Hard-coded number of samples for short-term and long-term time windows. - private final int numLongSamples = 1000; - private final int numShortSamples = 10; + // ZooKeeper belonging to the pulsar service. + private ZooKeeper zkClient; - // Initialize fields when they do not depend on PulsarService. + /** + * Initializes fields which do not depend on PulsarService. initialize(PulsarService) should subsequently be called. + */ public ModularLoadManagerImpl() { - loadData = new LoadData(); - preallocatedBundleToBroker = new ConcurrentHashMap<>(); brokerCandidateCache = new HashSet<>(); - primariesCache = new HashSet<>(); - sharedCache = new HashSet<>(); + defaultStats = new NamespaceBundleStats(); filterPipeline = new ArrayList<>(); + loadData = new LoadData(); loadSheddingPipeline = new ArrayList<>(); - defaultStats = new NamespaceBundleStats(); + preallocatedBundleToBroker = new ConcurrentHashMap<>(); + primariesCache = new HashSet<>(); scheduler = Executors.newScheduledThreadPool(1); + sharedCache = new HashSet<>(); } /** - * Initialize this load manager using the given PulsarService. + * 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) { - if (SystemUtils.IS_OS_LINUX) { - brokerHostUsage = new LinuxBrokerHostUsageImpl(pulsar); - } else { - brokerHostUsage = new GenericBrokerHostUsageImpl(pulsar); - } - this.pulsar = pulsar; - zkClient = pulsar.getZkClient(); - conf = pulsar.getConfiguration(); - policies = new SimpleResourceAllocationPolicies(pulsar); - localData = new LocalBrokerData(pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), - pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()); - placementStrategy = ModularLoadManagerStrategy.create(conf); adminCache = CacheBuilder.newBuilder().removalListener(new RemovalListener() { public void onRemoval(RemovalNotification removal) { removal.getValue().close(); @@ -167,19 +196,6 @@ public PulsarAdmin load(String key) throws Exception { } }); - // Initialize the default stats to assume for unseen bundles (hard-coded for now). - defaultStats.msgThroughputIn = 50000; - defaultStats.msgThroughputOut = 50000; - defaultStats.msgRateIn = 50; - defaultStats.msgRateOut = 50; - - 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); availableActiveBrokers = new ZooKeeperChildrenCache(pulsar.getLocalZkCache(), LoadManager.LOADBALANCE_BROKERS_ROOT); availableActiveBrokers.registerListener(new ZooKeeperCacheListener>() { @@ -191,6 +207,36 @@ public void onUpdate(String path, Set data, Stat stat) { 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(); } /** @@ -204,10 +250,73 @@ public ModularLoadManagerImpl(final PulsarService pulsar) { initialize(pulsar); } - /* - * 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. - */ + // 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(); @@ -235,26 +344,9 @@ private void updateAllBrokerData() { } } - /* - * Use the Pulsar client to acquire the namespace bundle stats. - */ - private Map getBundleStats() { - return pulsar.getBrokerService().getBundleStats(); - } - - /** - * Update both the broker data and the bundle data. - */ - public void updateAll() { - updateAllBrokerData(); - updateBundleData(); - } - - /** - * 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. - */ - public void updateBundleData() { + // 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()) { @@ -301,84 +393,6 @@ public void updateBundleData() { } } - // 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()); - } - - // 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. - } - } - } - - // Get the ZooKeeper path for the given bundle full name. - public static String getBundleDataZooKeeperPath(final String bundle) { - return BUNDLE_DATA_ZPATH + "/" + bundle; - } - - // Get the total number of used bytes in the JVM. - private static long getRealtimeJVMHeapUsageBytes() { - return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); - } - - // Get the system resource usage for this broker. - 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(); - long memoryUsageInBytes = getRealtimeJVMHeapUsageBytes(); - 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; - } - - // 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); - } - - // 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(numShortSamples, numLongSamples, defaultStats); - } - return bundleData; - } - /** * As any broker, disable the broker this manager is running on. * @@ -396,20 +410,6 @@ public void disableBroker() throws PulsarServerException { } } - 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()); - } - /** * As the leader broker, select bundles for the namespace service to unload so that they may be reassigned to new * brokers. @@ -424,7 +424,8 @@ public synchronized void doLoadShedding() { final String bundle = entry.getKey(); final String broker = entry.getValue(); adminCache.get(broker).namespaces().unloadNamespaceBundle( - getNamespaceNameFromBundleName(bundle), getBundleRangeFromBundleName(bundle)); + LoadManagerShared.getNamespaceNameFromBundleName(bundle), + LoadManagerShared.getBundleRangeFromBundleName(bundle)); } } catch (Exception e) { log.warn("Error when trying to perform load shedding: {}", e); @@ -472,7 +473,7 @@ public synchronized String selectBrokerForAssignment(final ServiceUnitId service final BundleData data = loadData.getBundleData().computeIfAbsent(bundle, key -> getBundleDataOrDefault(bundle)); brokerCandidateCache.clear(); brokerCandidateCache.addAll(loadData.getBrokerData().keySet()); - policyFilter(serviceUnit); + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache); // Use the filter pipeline to finalize broker candidates. for (BrokerFilter filter : filterPipeline) { @@ -486,75 +487,6 @@ public synchronized String selectBrokerForAssignment(final ServiceUnitId service return broker; } - private void policyFilter(final ServiceUnitId serviceUnit) { - 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 : brokerCandidateCache) { - 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); - } - } - /** * As any broker, start the load manager. * @@ -606,7 +538,7 @@ public void stop() throws PulsarServerException { @Override public void updateLocalBrokerData() { try { - final SystemResourceUsage systemResourceUsage = getSystemResourceUsage(); + final SystemResourceUsage systemResourceUsage = LoadManagerShared.getSystemResourceUsage(brokerHostUsage); localData.update(systemResourceUsage, getBundleStats()); } catch (Exception e) { log.warn("Error when attempting to update local broker data: {}", e); 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 839862760b3b9..904ce824bbafc 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 @@ -97,6 +97,8 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private long brokerRotationCursor = 0; // 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; // Caches for bundle gains and losses. private final Set bundleGainsCache; @@ -185,6 +187,7 @@ public SimpleLoadManagerImpl() { placementStrategy = new WRRPlacementStrategy(); bundleGainsCache = new HashSet<>(); bundleLossesCache = new HashSet<>(); + brokerCandidateCache = new HashSet<>(); } @Override @@ -871,93 +874,29 @@ private synchronized ResourceUnit findBrokerForPlacement(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; + synchronized (brokerCandidateCache) { + brokerCandidateCache.clear(); + for (final Set resourceUnitSet : availableBrokers.values()) { + for (final ResourceUnit resourceUnit : resourceUnitSet) { + brokerCandidateCache.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()); - } + } - } - } 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()); + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache); + final Multimap result = TreeMultimap.create(); + // 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; } } @@ -1099,19 +1038,9 @@ 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; } @@ -1128,10 +1057,10 @@ public LoadReport generateLoadReport() throws Exception { 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.setOverLoaded(isAboveLoadLevel(systemResourceUsage, + this.getLoadBalancerBrokerOverloadedThresholdPercentage())); + loadReport.setUnderLoaded(isBelowLoadLevel(systemResourceUsage, + this.getLoadBalancerBrokerUnderloadedThresholdPercentage())); loadReport.setSystemResourceUsage(systemResourceUsage); loadReport.setBundleStats(pulsar.getBrokerService().getBundleStats()); @@ -1165,8 +1094,8 @@ public LoadReport generateLoadReport() throws Exception { loadReport.setAllocatedMsgRateIn(allocatedQuota.getMsgRateIn()); loadReport.setAllocatedMsgRateOut(allocatedQuota.getMsgRateOut()); - final ResourceUnit resourceUnit = new SimpleResourceUnit(String.format("http://%s", loadReport.getName()), - fromLoadReport(loadReport)); + final ResourceUnit resourceUnit = new SimpleResourceUnit( + String.format("http://%s", loadReport.getName()), fromLoadReport(loadReport)); Set preAllocatedBundles; if (resourceUnitRankings.containsKey(resourceUnit)) { preAllocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles(); @@ -1287,23 +1216,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); @@ -1349,7 +1264,8 @@ 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 " @@ -1449,7 +1365,8 @@ public void doNamespaceBundleSplit() throws Exception { 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.", @@ -1478,7 +1395,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); From 1ea109e6dedb7288a41d53cd1a1c06baa2119892 Mon Sep 17 00:00:00 2001 From: breese Date: Tue, 28 Mar 2017 16:59:08 -0700 Subject: [PATCH 17/26] Remove redundant synchronized block, make correction to onUpdate --- .../impl/ModularLoadManagerImpl.java | 2 +- .../impl/SimpleLoadManagerImpl.java | 98 +++++++++---------- 2 files changed, 50 insertions(+), 50 deletions(-) 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 index 415ac705e07d9..33bdece8cc607 100644 --- 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 @@ -204,7 +204,7 @@ public void onUpdate(String path, Set data, Stat stat) { if (log.isDebugEnabled()) { log.debug("Update Received for path {}", path); } - scheduler.submit(ModularLoadManagerImpl.this::updateAll); + scheduler.submit(ModularLoadManagerImpl.this::updateLocalBrokerData); } }); 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 904ce824bbafc..05eb56e6b8650 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 @@ -666,59 +666,59 @@ 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(), - loadedBundles, allocatedQuota, preAllocatedBundles, preAllocatedQuota); - 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 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); - } + Map> newSortedRankings = Maps.newTreeMap(); + Map newResourceUnitRankings = new HashMap<>(); - // update metrics - if (resourceUnit.getResourceId().contains(hostname)) { - updateLoadBalancingMetrics(hostname, finalRank, ranking); - } + 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); + + // 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); + } + + // 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()); From 31d60cb88aafeaf502e5c4a815b2d45e626790b7 Mon Sep 17 00:00:00 2001 From: breese Date: Tue, 28 Mar 2017 21:54:33 -0700 Subject: [PATCH 18/26] Fix policy filter bug --- .../loadbalance/impl/LoadManagerShared.java | 5 +- .../impl/ModularLoadManagerImpl.java | 5 +- .../impl/SimpleLoadManagerImpl.java | 148 +++++++++--------- 3 files changed, 80 insertions(+), 78 deletions(-) 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 index 816db7c2794e0..ee99cba545eef 100644 --- 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 @@ -52,7 +52,8 @@ 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 SimpleResourceAllocationPolicies policies, final Set brokerCandidateCache, + final Set availableBrokers) { primariesCache.clear(); sharedCache.clear(); NamespaceName namespace = serviceUnit.getNamespaceObject(); @@ -60,7 +61,7 @@ public static synchronized void applyPolicies(final ServiceUnitId serviceUnit, if (isIsolationPoliciesPresent) { log.debug("Isolation Policies Present for namespace - [{}]", namespace.toString()); } - for (final String broker : brokerCandidateCache) { + for (final String broker : availableBrokers) { final String brokerUrlString = String.format("http://%s", broker); URL brokerUrl; try { 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 index 33bdece8cc607..45cbc23a4f724 100644 --- 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 @@ -204,7 +204,7 @@ public void onUpdate(String path, Set data, Stat stat) { if (log.isDebugEnabled()) { log.debug("Update Received for path {}", path); } - scheduler.submit(ModularLoadManagerImpl.this::updateLocalBrokerData); + scheduler.submit(ModularLoadManagerImpl.this::updateAll); } }); @@ -472,8 +472,7 @@ public synchronized String selectBrokerForAssignment(final ServiceUnitId service } final BundleData data = loadData.getBundleData().computeIfAbsent(bundle, key -> getBundleDataOrDefault(bundle)); brokerCandidateCache.clear(); - brokerCandidateCache.addAll(loadData.getBrokerData().keySet()); - LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache); + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, loadData.getBrokerData().keySet()); // Use the filter pipeline to finalize broker candidates. for (BrokerFilter filter : filterPipeline) { 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 05eb56e6b8650..3320c2da1d4c9 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 @@ -786,89 +786,87 @@ private synchronized ResourceUnit findBrokerForPlacement(Multimap 0) ? (this.brokerRotationCursor % candidates.size()) : 0; + 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--; + // 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; - } + // 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 ((unboundedRanks ? ranking.compareMessageRateTo(selectedRanking) - : 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 && !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; - } + 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); - resourceUnitRankings.put(selectedRU, ranking); - } + // 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; @@ -877,15 +875,19 @@ private synchronized ResourceUnit findBrokerForPlacement(Multimap getFinalCandidates(ServiceUnitId serviceUnit, Map> availableBrokers) { synchronized (brokerCandidateCache) { + final Multimap result = TreeMultimap.create(); brokerCandidateCache.clear(); - for (final Set resourceUnitSet : availableBrokers.values()) { - for (final ResourceUnit resourceUnit : resourceUnitSet) { - brokerCandidateCache.add(resourceUnit.getResourceId().replace("http://", "")); + try { + LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, + availableActiveBrokers.get()); + } 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; } - LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache); - final Multimap result = TreeMultimap.create(); // 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(); From 14100081bd3254d52415c0aefebad043481adbac Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 29 Mar 2017 10:01:27 -0700 Subject: [PATCH 19/26] Remove availableActiveBrokers.get() call --- .../loadbalance/impl/SimpleLoadManagerImpl.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) 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 3320c2da1d4c9..bf8bf4daf441f 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 @@ -97,9 +97,13 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private long brokerRotationCursor = 0; // 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; @@ -188,6 +192,7 @@ public SimpleLoadManagerImpl() { bundleGainsCache = new HashSet<>(); bundleLossesCache = new HashSet<>(); brokerCandidateCache = new HashSet<>(); + availableBrokersCache = new HashSet<>(); } @Override @@ -876,13 +881,19 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni 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://", "")); + } + } brokerCandidateCache.clear(); try { LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, - availableActiveBrokers.get()); + availableBrokersCache); } catch (Exception e) { log.warn("Error when trying to apply policies: {}", e); - for (final Map.Entry> entry: availableBrokers.entrySet()) { + for (final Map.Entry> entry : availableBrokers.entrySet()) { result.putAll(entry.getKey(), entry.getValue()); } return result; From 92028f99fdd903620ed701d33068b322dab25d35 Mon Sep 17 00:00:00 2001 From: breese Date: Wed, 29 Mar 2017 14:11:37 -0700 Subject: [PATCH 20/26] Use connection pooling --- .../pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java | 3 --- .../pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java | 3 +-- .../java/com/yahoo/pulsar/testclient/LoadSimulationClient.java | 3 +-- 3 files changed, 2 insertions(+), 7 deletions(-) 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 index 45cbc23a4f724..d4ef9e8696e29 100644 --- 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 @@ -83,9 +83,6 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach // Note that the default message size is implicitly defined as DEFAULT_MESSAGE_THROUGHPUT / DEFAULT_MESSAGE_RATE. public static final double DEFAULT_MESSAGE_THROUGHPUT = 50000; - // Value of prefix "mibi" (e.g., number of bytes in a mibibyte). - public static final int MIBI = 1024 * 1024; - // The number of effective samples to keep for observing long term data. public static final int NUM_LONG_SAMPLES = 1000; 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 bf8bf4daf441f..fe830fca4d3bb 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 @@ -889,8 +889,7 @@ private Multimap getFinalCandidates(ServiceUnitId serviceUni } brokerCandidateCache.clear(); try { - LoadManagerShared.applyPolicies(serviceUnit, policies, brokerCandidateCache, - availableBrokersCache); + 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()) { 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 index 25eede82a1e03..b61420ce1009e 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java @@ -314,8 +314,7 @@ public LoadSimulationClient(final MainArguments arguments) throws Exception { new DefaultThreadFactory("pulsar-test-client")); clientConf = new ClientConfiguration(); - // Disable connection pooling. - clientConf.setConnectionsPerBroker(0); + clientConf.setConnectionsPerBroker(4); // Disable stats on the clients to reduce CPU/memory usage. clientConf.setStatsInterval(0, TimeUnit.SECONDS); From ba9c03280777a64a8dce8f7b026dc607cbb24730 Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 30 Mar 2017 10:15:12 -0700 Subject: [PATCH 21/26] Remove unused code --- .../pulsar/broker/ServiceConfiguration.java | 29 ++----------------- .../impl/SimpleLoadManagerImpl.java | 6 +--- 2 files changed, 4 insertions(+), 31 deletions(-) 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 2fd7dc6c287de..7039a1a630a1f 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 @@ -200,9 +200,6 @@ public class ServiceConfiguration implements PulsarConfiguration { // load placement strategy private String loadBalancerPlacementStrategy = "weightedRandomSelection"; // weighted random selection // load placement secondary strategy (used to silently test an alternate strategy) - private String loadBalancerSecondaryStrategy = null; - // are all bundle placement operations forwarded to a lead broker - private boolean loadBalancerIsCentralized = false; // Percentage of change to trigger load report update private int loadBalancerReportUpdateThresholdPercentage = 10; // maximum interval to update load report @@ -259,6 +256,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; @@ -739,22 +739,6 @@ public String getLoadBalancerPlacementStrategy() { return this.loadBalancerPlacementStrategy; } - public void setLoadBalancerSecondaryStrategy(String secondaryStrategy) { - this.loadBalancerSecondaryStrategy = secondaryStrategy; - } - - public String getLoadBalancerSecondaryStrategy() { - return this.loadBalancerSecondaryStrategy; - } - - public void setLoadBalancerIsCentralized(boolean isCentralized) { - this.loadBalancerIsCentralized = isCentralized; - } - - public boolean getLoadBalancerIsCentralized() { - return this.loadBalancerIsCentralized; - } - public int getLoadBalancerReportUpdateThresholdPercentage() { return loadBalancerReportUpdateThresholdPercentage; } @@ -973,13 +957,6 @@ public void setReplicatorPrefix(String replicatorPrefix) { this.replicatorPrefix = replicatorPrefix; } - - // Configurations for new load manager API - - // Name of load manager to use - @FieldContext(dynamic = true) - private String loadManagerClassName = "com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl"; - public String getLoadManagerClassName() { return loadManagerClassName; } 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 fe830fca4d3bb..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 @@ -145,7 +145,6 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private LoadingCache unloadedHotNamespaceCache; public static final String LOADBALANCER_DYNAMIC_SETTING_STRATEGY_ZPATH = "/loadbalance/settings/strategy"; - public static final String LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH = "/loadbalance/settings/is_centralized"; 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"; private static final String LOADBALANCER_DYNAMIC_SETTING_OVERLOAD_THRESHOLD_ZPATH = "/loadbalance/settings/overload_threshold"; @@ -155,7 +154,6 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene private static final String SETTING_NAME_LOAD_FACTOR_CPU = "loadFactorCPU"; private static final String SETTING_NAME_LOAD_FACTOR_MEM = "loadFactorMemory"; private static final String SETTING_NAME_STRATEGY = "loadBalancerStrategy"; - private static final String SETTING_NAME_IS_CENTRALIZED = "loadBalancerIsCentralized"; private static final String SETTING_NAME_OVERLOAD_THRESHOLD = "overloadThreshold"; private static final String SETTING_NAME_UNDERLOAD_THRESHOLD = "underloadThreshold"; private static final String SETTING_NAME_COMFORTLOAD_THRESHOLD = "comfortLoadThreshold"; @@ -401,9 +399,7 @@ public boolean isCentralized() { if (strategy.equals(LOADBALANCER_STRATEGY_LLS) || strategy.equals(LOADBALANCER_STRATEGY_LEAST_MSG)) { return true; } - - return this.getDynamicConfigurationBoolean(LOADBALANCER_DYNAMIC_SETTING_CENTRALIZED_ZPATH, - SETTING_NAME_IS_CENTRALIZED, pulsar.getConfiguration().getLoadBalancerIsCentralized()); + return false; } private long getLoadBalancerBrokerUnderloadedThresholdPercentage() { From 348aba3657ab049afdcf3f7a9d0eefc265a56f45 Mon Sep 17 00:00:00 2001 From: Brad McMillen Date: Thu, 30 Mar 2017 12:58:42 -0700 Subject: [PATCH 22/26] Don't try reading the symlink and converting to absolute path --- .../impl/LinuxBrokerHostUsageImpl.java | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) 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 6092b516c15c9..0c8a584d61b48 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 @@ -155,22 +155,12 @@ private List getNics() { private boolean isPhysicalNic(Path path) { try { - path = path.toAbsolutePath(); - 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; - } - } - } catch (IOException e) { - LOG.error("Failed to read link target for NIC " + path, e); + Files.readAllBytes(path.resolve("speed")); + return true; + } catch (Exception e) { + // wireless nics don't report speed, ignore them. return false; } - return false; } private Path getNicSpeedPath(String nic) { From 1efe869ca4656ac381b41cbe12893956177450b7 Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 30 Mar 2017 13:20:27 -0700 Subject: [PATCH 23/26] Remove redundant and unnecessary statments --- bin/pulsar-perf | 18 +++++++++--------- .../pulsar/broker/ServiceConfiguration.java | 1 - .../pulsar/broker/service/BrokerService.java | 1 - 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/bin/pulsar-perf b/bin/pulsar-perf index 83b9715850874..5794da2b35013 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -73,14 +73,14 @@ pulsar_help() { cat < where command is one of: - produce Run a producer - consume Run a consumer - simple-monitor Continuously receive broker data when using SimpleLoadManagerImpl - modular-monitor 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 + 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 + help This help message or command is the full name of a class with a defined main() method. @@ -142,9 +142,9 @@ 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" == "simple-monitor" ]; then +elif [ "$COMMAND" == "monitor-simple-load-manager" ]; then exec $JAVA $OPTS com.yahoo.pulsar.testclient.SimpleLoadManagerBrokerMonitor "$@" -elif [ "$COMMAND" == "modular-monitor" ]; then +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 "$@" 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 7039a1a630a1f..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 @@ -199,7 +199,6 @@ public class ServiceConfiguration implements PulsarConfiguration { private boolean loadBalancerEnabled = false; // load placement strategy private String loadBalancerPlacementStrategy = "weightedRandomSelection"; // weighted random selection - // load placement secondary strategy (used to silently test an alternate strategy) // Percentage of change to trigger load report update private int loadBalancerReportUpdateThresholdPercentage = 10; // maximum interval to update load report 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 1c56d7baa8b24..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 @@ -913,7 +913,6 @@ private void updateConfigurationAndRegisterListeners() { (maxConcurrentTopicLoadRequest) -> topicLoadRequestSemaphore.set(new Semaphore((int) maxConcurrentTopicLoadRequest, false))); registerConfigurationListener("loadManagerClassName", className -> { try { - log.info("Attempting to change load manager"); final LoadManager newLoadManager = LoadManager.create(pulsar); log.info("Created load manager: {}", className); pulsar.getLoadManager().get().disableBroker(); From 96c0d86e58d2041e0cfc9cf7fbfa0a875a63ad14 Mon Sep 17 00:00:00 2001 From: Brad McMillen Date: Thu, 30 Mar 2017 14:04:37 -0700 Subject: [PATCH 24/26] Add test back to see if path contains /virtual/ --- .../impl/LinuxBrokerHostUsageImpl.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 0c8a584d61b48..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 @@ -154,13 +154,16 @@ private List getNics() { } private boolean isPhysicalNic(Path path) { - 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; } private Path getNicSpeedPath(String nic) { From 9e39cc7bba8fc100ab3424c1cad9b3c5db56e374 Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 30 Mar 2017 14:12:23 -0700 Subject: [PATCH 25/26] Replace System.out statements with log.info --- .../testclient/LoadSimulationClient.java | 8 ++- .../testclient/LoadSimulationController.java | 59 ++++++++++--------- .../ModularLoadManagerBrokerMonitor.java | 47 ++++++++------- .../SimpleLoadManagerBrokerMonitor.java | 57 +++++++++--------- 4 files changed, 89 insertions(+), 82 deletions(-) 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 index b61420ce1009e..01513b7e8516d 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java @@ -49,12 +49,16 @@ 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; @@ -355,9 +359,9 @@ public void run() throws Exception { // Technically, two controllers can be connected simultaneously, but // non-sequential handling of commands // has not been tested or considered and is not recommended. - System.out.println("Listening for controller command..."); + log.info("Listening for controller command..."); final Socket socket = serverSocket.accept(); - System.out.format("Connected to %s\n", socket.getInetAddress().getHostName()); + log.info("Connected to %s\n", socket.getInetAddress().getHostName()); executor.submit(() -> { try { handle(socket); 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 index a907c32167eb0..4786009431518 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationController.java @@ -44,6 +44,8 @@ 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. @@ -60,6 +62,7 @@ * */ 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. @@ -207,7 +210,7 @@ public synchronized void process(final WatchedEvent event) { /** * Create a LoadSimulationController with the given JCommander arguments. - * + * * @param arguments * Arguments to create from. */ @@ -219,12 +222,12 @@ public LoadSimulationController(final MainArguments arguments) throws Exception final Socket[] sockets = new Socket[servers.length]; inputStreams = new DataInputStream[servers.length]; outputStreams = new DataOutputStream[servers.length]; - System.out.format("Found %d servers:\n", 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()); - System.out.format("Connected to %s\n", servers[i]); + log.info("Connected to {}", servers[i]); } } @@ -232,7 +235,7 @@ public LoadSimulationController(final MainArguments arguments) throws Exception // actual number of application arguments. private boolean checkAppArgs(final int numAppArgs, final int numRequired) { if (numAppArgs != numRequired) { - System.out.format("ERROR: Wrong number of application arguments (found %d, required %d)\n", numAppArgs, + log.info("ERROR: Wrong number of application arguments (found {}, required {})", numAppArgs, numRequired); return false; } @@ -254,7 +257,7 @@ private void writeProducerOptions(final DataOutputStream outputStream, final She // doubles. final String[] splits = arguments.rangeString.split(","); if (splits.length != 2) { - System.out.println("ERROR: Argument to --rand-rate should be a two comma-separated values"); + log.info("ERROR: Argument to --rand-rate should be a two comma-separated values"); return; } final double first = Double.parseDouble(splits[0]); @@ -273,14 +276,14 @@ private synchronized void trade(final ShellArguments arguments, final String des // Decide which server to send to randomly to preserve statelessness of // the controller. final int i = random.nextInt(servers.length); - System.out.println("Sending trade request to " + servers[i]); + 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) { - System.out.println("Created producer and consumer for " + destination); + log.info("Created producer and consumer for " + destination); } else { - System.out.format("ERROR: Socket to %s closed\n", servers[i]); + log.info("ERROR: Socket to {} closed", servers[i]); } } @@ -299,7 +302,7 @@ private void handleTrade(final ShellArguments arguments) throws Exception { // arguments. // Returns true if the topic was found and false otherwise. private synchronized boolean change(final ShellArguments arguments, final String destination) throws Exception { - System.out.println("Searching for server with topic " + destination); + log.info("Searching for server with topic " + destination); for (DataOutputStream outputStream : outputStreams) { outputStream.write(LoadSimulationClient.CHANGE_COMMAND); writeProducerOptions(outputStream, arguments, destination); @@ -310,16 +313,16 @@ private synchronized boolean change(final ShellArguments arguments, final String int readValue; switch (readValue = inputStreams[i].read()) { case LoadSimulationClient.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); + log.info("Found topic {} on server {}", destination, servers[i]); foundTopic = true; break; case LoadSimulationClient.NO_SUCH_TOPIC: break; case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); + log.info("ERROR: Socket to {} closed", servers[i]); break; default: - System.out.println("ERROR: Unknown response signal received: " + readValue); + log.info("ERROR: Unknown response signal received: " + readValue); } } return foundTopic; @@ -333,7 +336,7 @@ private void handleChange(final ShellArguments arguments) throws Exception { final String destination = makeDestination(commandArguments.get(1), commandArguments.get(2), commandArguments.get(3)); if (!change(arguments, destination)) { - System.out.format("ERROR: Topic %s not found\n", destination); + log.info("ERROR: Topic {} not found", destination); } } } @@ -345,7 +348,7 @@ private void handleStop(final ShellArguments arguments) throws Exception { if (checkAppArgs(commandArguments.size() - 1, 3)) { final String destination = makeDestination(commandArguments.get(1), commandArguments.get(2), commandArguments.get(3)); - System.out.println("Searching for server with topic " + destination); + log.info("Searching for server with topic " + destination); for (DataOutputStream outputStream : outputStreams) { outputStream.write(LoadSimulationClient.STOP_COMMAND); outputStream.writeUTF(destination); @@ -356,24 +359,24 @@ private void handleStop(final ShellArguments arguments) throws Exception { int readValue; switch (readValue = inputStreams[i].read()) { case LoadSimulationClient.FOUND_TOPIC: - System.out.format("Found topic %s on server %s\n", destination, servers[i]); + log.info("Found topic {} on server {}", destination, servers[i]); foundTopic = true; break; case LoadSimulationClient.NO_SUCH_TOPIC: break; case LoadSimulationClient.REDUNDANT_COMMAND: - System.out.format("ERROR: Topic %s already stopped on %s\n", destination, servers[i]); + log.info("ERROR: Topic {} already stopped on {}", destination, servers[i]); foundTopic = true; break; case -1: - System.out.format("ERROR: Socket to %s closed\n", servers[i]); + log.info("ERROR: Socket to {} closed", servers[i]); break; default: - System.out.println("ERROR: Unknown response signal received: " + readValue); + log.info("ERROR: Unknown response signal received: " + readValue); } } if (!foundTopic) { - System.out.format("ERROR: Topic %s not found\n", destination); + log.info("ERROR: Topic {} not found", destination); } } } @@ -427,23 +430,23 @@ private void accumulateAndReport(final String tenant, final String group) throws for (int i = 0; i < servers.length; ++i) { final int foundOnServer = inputStreams[i].readInt(); if (foundOnServer == -1) { - System.out.format("ERROR: Socket to %s closed\n", servers[i]); + log.info("ERROR: Socket to {} closed", servers[i]); } else if (foundOnServer == 0) { - System.out.format("Found no topics belonging to tenant %s and group %s on %s\n", tenant, group, + log.info("Found no topics belonging to tenant {} and group {} on {}", tenant, group, servers[i]); } else if (foundOnServer > 0) { - System.out.format("Found %d topics belonging to tenant %s and group %s on %s\n", foundOnServer, tenant, + log.info("Found {} topics belonging to tenant {} and group {} on {}", foundOnServer, tenant, group, servers[i]); numFound += foundOnServer; } else { - System.out.format("ERROR: Negative value %d received for topic count on %s\n", foundOnServer, + log.info("ERROR: Negative value {} received for topic count on {}", foundOnServer, servers[i]); } } if (numFound == 0) { - System.out.format("ERROR: Found no topics belonging to tenant %s and group %s\n", tenant, group); + log.info("ERROR: Found no topics belonging to tenant {} and group {}", tenant, group); } else { - System.out.format("Found %d topics belonging to tenant %s and group %s\n", numFound, tenant, group); + log.info("Found {} topics belonging to tenant {} and group {}", numFound, tenant, group); } } @@ -554,11 +557,11 @@ private void handleCopy(final ShellArguments arguments) throws Exception { mangledNamespace); final String newAPITargetPath = String.format("/loadbalance/bundle-data/%s/%s/%s/0x00000000_0xffffffff", cluster, tenantName, mangledNamespace); - System.out.format("Copying %s to %s\n", bundle, oldAPITargetPath); + log.info("Copying {} to {}", bundle, oldAPITargetPath); ZkUtils.createFullPathOptimistic(targetZKClient, oldAPITargetPath, ObjectMapperFactory.getThreadLocal().writeValueAsBytes(quota), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - System.out.format("Creating new API data at %s\n", newAPITargetPath); + 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); @@ -622,7 +625,7 @@ public void read(final String[] args) { System.exit(0); break; default: - System.out.format("ERROR: Unknown command \"%s\"\n", command); + log.info("ERROR: Unknown command \"{}\"", command); } } catch (ParameterException ex) { ex.printStackTrace(); 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 index ce083c2b22695..afdb32d3d1712 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/ModularLoadManagerBrokerMonitor.java @@ -29,8 +29,11 @@ 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; @@ -64,12 +67,12 @@ public synchronized void updateBrokers(final String path) { } for (String oldBroker : brokers) { if (!newBrokers.contains(oldBroker)) { - System.out.println("Lost broker: " + oldBroker); + log.info("Lost broker: " + oldBroker); } } for (String newBroker : newBrokers) { if (!brokers.contains(newBroker)) { - System.out.println("Gained broker: " + newBroker); + log.info("Gained broker: " + newBroker); final BrokerDataWatcher brokerDataWatcher = new BrokerDataWatcher(zkClient); brokerDataWatcher.printBrokerData(path + "/" + newBroker); } @@ -102,10 +105,10 @@ public synchronized void process(final WatchedEvent event) { private static void printMessageData(final double msgThroughputIn, final double msgThroughputOut, final double msgRateIn, final double msgRateOut) { - System.out.format("Message Throughput In: %.2f KB/s\n", msgThroughputIn / 1024); - System.out.format("Message Throughput Out: %.2f KB/s\n", msgThroughputOut / 1024); - System.out.format("Message Rate In: %.2f msgs/s\n", msgRateIn); - System.out.format("Message Rate Out: %.2f msgs/s\n", 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) { @@ -119,22 +122,21 @@ public synchronized void printBrokerData(final String brokerPath) { throw new RuntimeException(ex); } - System.out.println("\nBroker Data for " + broker + ":"); - System.out.println("---------------"); + log.info("Broker Data for " + broker + ":"); + log.info("---------------"); - System.out.println("\nNum Topics: " + localBrokerData.getNumTopics()); - System.out.println("Num Bundles: " + localBrokerData.getNumBundles()); - System.out.println("Num Consumers: " + localBrokerData.getNumConsumers()); - System.out.println("Num Producers: " + localBrokerData.getNumProducers()); + log.info("Num Topics: " + localBrokerData.getNumTopics()); + log.info("Num Bundles: " + localBrokerData.getNumBundles()); + log.info("Num Consumers: " + localBrokerData.getNumConsumers()); + log.info("Num Producers: " + localBrokerData.getNumProducers()); - System.out.println(String.format("\nCPU: %.2f%%", localBrokerData.getCpu().percentUsage())); + log.info(String.format("CPU: %.2f%%", localBrokerData.getCpu().percentUsage())); - System.out.println(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); + log.info(String.format("Memory: %.2f%%", localBrokerData.getMemory().percentUsage())); - System.out - .println(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); + log.info(String.format("Direct Memory: %.2f%%", localBrokerData.getDirectMemory().percentUsage())); - System.out.println("\nLatest Data:\n"); + log.info("Latest Data:"); printMessageData(localBrokerData.getMsgThroughputIn(), localBrokerData.getMsgThroughputOut(), localBrokerData.getMsgRateIn(), localBrokerData.getMsgRateOut()); @@ -145,28 +147,25 @@ public synchronized void printBrokerData(final String brokerPath) { } catch (Exception ex) { throw new RuntimeException(ex); } - System.out.println("\nShort Term Data:\n"); + log.info("Short Term Data:"); printMessageData(timeAverageData.getShortTermMsgThroughputIn(), timeAverageData.getShortTermMsgThroughputOut(), timeAverageData.getShortTermMsgRateIn(), timeAverageData.getShortTermMsgRateOut()); - System.out.println("\nLong Term Data:\n"); + log.info("Long Term Data:"); printMessageData(timeAverageData.getLongTermMsgThroughputIn(), timeAverageData.getLongTermMsgThroughputOut(), timeAverageData.getLongTermMsgRateIn(), timeAverageData.getLongTermMsgRateOut()); - System.out.println(); if (!localBrokerData.getLastBundleGains().isEmpty()) { for (String bundle : localBrokerData.getLastBundleGains()) { - System.out.println("Gained Bundle: " + bundle); + log.info("Gained Bundle: " + bundle); } - System.out.println(); } if (!localBrokerData.getLastBundleLosses().isEmpty()) { for (String bundle : localBrokerData.getLastBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); + log.info("Lost Bundle: " + bundle); } - System.out.println(); } } } 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 index 39827e3b6b746..603d8b82d3009 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/SimpleLoadManagerBrokerMonitor.java @@ -23,6 +23,8 @@ 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.*; @@ -31,6 +33,7 @@ * 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; @@ -64,12 +67,12 @@ public synchronized void updateBrokers(final String path) { } for (String oldBroker : brokers) { if (!newBrokers.contains(oldBroker)) { - System.out.println("Lost broker: " + oldBroker); + log.info("Lost broker: " + oldBroker); } } for (String newBroker : newBrokers) { if (!brokers.contains(newBroker)) { - System.out.println("Gained broker: " + newBroker); + log.info("Gained broker: " + newBroker); final LoadReportWatcher loadReportWatcher = new LoadReportWatcher(zkClient); loadReportWatcher.printLoadReport(path + "/" + newBroker); } @@ -105,54 +108,52 @@ public synchronized void printLoadReport(final String path) { } final SystemResourceUsage resourceUsage = loadReport.getSystemResourceUsage(); - System.out.println("\nLoad Report for " + brokerName + ":"); - System.out.println("---------------"); + log.info("Load Report for " + brokerName + ":"); + log.info("---------------"); - System.out.println("\nNum Topics: " + loadReport.getNumTopics()); - System.out.println("Num Bundles: " + loadReport.getNumBundles()); + log.info("Num Topics: " + loadReport.getNumTopics()); + log.info("Num Bundles: " + loadReport.getNumBundles()); - System.out.format("\nRaw CPU: %.2f%%\n", resourceUsage.getCpu().percentUsage()); - System.out.println(String.format("Allocated CPU: %.2f%%", + log.info(String.format("Raw CPU: %.2f%%", resourceUsage.getCpu().percentUsage())); + log.info(String.format("Allocated CPU: %.2f%%", percentUsage(loadReport.getAllocatedCPU(), resourceUsage.getCpu().limit))); - System.out.println(String.format("Preallocated CPU: %.2f%%", + log.info(String.format("Preallocated CPU: %.2f%%", percentUsage(loadReport.getPreAllocatedCPU(), resourceUsage.getCpu().limit))); - System.out.format("\nRaw Memory: %.2f%%\n", resourceUsage.getMemory().percentUsage()); - System.out.println(String.format("Allocated Memory: %.2f%%", + log.info(String.format("Raw Memory: %.2f%%", resourceUsage.getMemory().percentUsage())); + log.info(String.format("Allocated Memory: %.2f%%", percentUsage(loadReport.getAllocatedMemory(), resourceUsage.getMemory().limit))); - System.out.println(String.format("Preallocated Memory: %.2f%%", + log.info(String.format("Preallocated Memory: %.2f%%", percentUsage(loadReport.getPreAllocatedMemory(), resourceUsage.getMemory().limit))); - System.out.format("\nRaw Bandwidth In: %.2f%%\n", resourceUsage.getBandwidthIn().percentUsage()); - System.out.println(String.format("Allocated Bandwidth In: %.2f%%", + 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))); - System.out.println(String.format("Preallocated Bandwidth In: %.2f%%", + log.info(String.format("Preallocated Bandwidth In: %.2f%%", percentUsage(loadReport.getPreAllocatedBandwidthIn(), resourceUsage.getBandwidthIn().limit))); - System.out.format("\nRaw Bandwidth Out: %.2f%%\n", resourceUsage.getBandwidthOut().percentUsage()); - System.out.println(String.format("Allocated Bandwidth Out: %.2f%%", + 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))); - System.out.println(String.format("Preallocated Bandwidth Out: %.2f%%", + log.info(String.format("Preallocated Bandwidth Out: %.2f%%", percentUsage(loadReport.getPreAllocatedBandwidthOut(), resourceUsage.getBandwidthOut().limit))); - System.out.format("\nDirect Memory: %.2f%%\n", resourceUsage.getDirectMemory().percentUsage()); + 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())); - System.out.format("Messages In Per Second: %.2f\n", loadReport.getMsgRateIn()); - System.out.format("Messages Out Per Second: %.2f\n", loadReport.getMsgRateOut()); - System.out.format("Preallocated Messages In Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateIn()); - System.out.format("Preallocated Out Per Second: %.2f\n", loadReport.getPreAllocatedMsgRateOut()); - System.out.println(); if (!loadReport.getBundleGains().isEmpty()) { for (String bundle : loadReport.getBundleGains()) { - System.out.println("Gained Bundle: " + bundle); + log.info("Gained Bundle: " + bundle); } - System.out.println(); } if (!loadReport.getBundleLosses().isEmpty()) { for (String bundle : loadReport.getBundleLosses()) { - System.out.println("Lost Bundle: " + bundle); + log.info("Lost Bundle: " + bundle); } - System.out.println(); } } } From 403ee0152ba3c185e54ca4c4596977727fc43106 Mon Sep 17 00:00:00 2001 From: breese Date: Thu, 30 Mar 2017 14:15:38 -0700 Subject: [PATCH 26/26] Change %s to {} --- .../java/com/yahoo/pulsar/testclient/LoadSimulationClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 01513b7e8516d..dae3492be97a5 100644 --- a/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/com/yahoo/pulsar/testclient/LoadSimulationClient.java @@ -361,7 +361,7 @@ public void run() throws Exception { // 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 %s\n", socket.getInetAddress().getHostName()); + log.info("Connected to {}\n", socket.getInetAddress().getHostName()); executor.submit(() -> { try { handle(socket);