Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve][broker] PIP-192 Added broker and top-bundles load reporters #19471

Merged
merged 3 commits into from
Feb 13, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2514,6 +2514,17 @@ The delayed message index bucket time step(in seconds) in per bucket snapshot se
)
private long loadBalancerBrokerLoadDataTTLInSeconds = 1800;

@FieldContext(
dynamic = true,
category = CATEGORY_LOAD_BALANCER,
doc = "Percentage of bundles to compute topK bundle load data from each broker. "
+ "The load balancer distributes bundles across brokers, "
+ "based on topK bundle load data and other broker load data."
+ "The bigger value will increase the overhead of reporting many bundles in load data. "
+ "(only used in load balancer extension logics)"
)
private double loadBalancerBundleLoadReportPercentage = 10;

/**** --- Replication. --- ****/
@FieldContext(
category = CATEGORY_REPLICATION,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
Expand All @@ -45,6 +47,8 @@
import org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision;
import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter;
import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision;
import org.apache.pulsar.broker.loadbalance.extensions.reporter.BrokerLoadDataReporter;
import org.apache.pulsar.broker.loadbalance.extensions.reporter.TopBundleLoadDataReporter;
import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore;
import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException;
import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory;
Expand Down Expand Up @@ -91,6 +95,16 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
@Getter
private final List<BrokerFilter> brokerFilterPipeline;

/**
* The load data reporter.
*/
private BrokerLoadDataReporter brokerLoadDataReporter;

private TopBundleLoadDataReporter topBundleLoadDataReporter;

private ScheduledFuture brokerLoadDataReportTask;
private ScheduledFuture topBundlesLoadDataReportTask;

private boolean started = false;

private final AssignCounter assignCounter = new AssignCounter();
Expand Down Expand Up @@ -147,7 +161,38 @@ public void start() throws PulsarServerException {
.brokerRegistry(brokerRegistry)
.brokerLoadDataStore(brokerLoadDataStore)
.topBundleLoadDataStore(topBundlesLoadDataStore).build();
// TODO: Start load data reporter.


this.brokerLoadDataReporter =
new BrokerLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), brokerLoadDataStore);

this.topBundleLoadDataReporter =
new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore);

var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis();
this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
.scheduleAtFixedRate(() -> {
try {
brokerLoadDataReporter.reportAsync(false);
// TODO: update broker load metrics using getLocalData
} catch (Throwable e) {
log.error("Failed to run the broker load manager executor job.", e);
}
},
interval,
interval, TimeUnit.MILLISECONDS);

this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor()
.scheduleAtFixedRate(() -> {
try {
// TODO: consider excluding the bundles that are in the process of split.
topBundleLoadDataReporter.reportAsync(false);
} catch (Throwable e) {
log.error("Failed to run the top bundles load manager executor job.", e);
}
},
interval,
interval, TimeUnit.MILLISECONDS);

// TODO: Start unload scheduler and bundle split scheduler
this.started = true;
Expand Down Expand Up @@ -264,6 +309,14 @@ public void close() throws PulsarServerException {
return;
}
try {
if (brokerLoadDataReportTask != null) {
brokerLoadDataReportTask.cancel(true);
}

if (topBundlesLoadDataReportTask != null) {
topBundlesLoadDataReportTask.cancel(true);
}

this.brokerLoadDataStore.close();
this.topBundlesLoadDataStore.close();
} catch (IOException ex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import java.util.List;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.Setter;
import lombok.ToString;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData;
Expand All @@ -37,6 +39,7 @@
*/
@Getter
@EqualsAndHashCode
@ToString
public class BrokerLoadData {

private static final double DEFAULT_RESOURCE_USAGE = 1.0d;
Expand All @@ -53,6 +56,7 @@ public class BrokerLoadData {
private double msgThroughputOut; // bytes/sec
private double msgRateIn; // messages/sec
private double msgRateOut; // messages/sec
private int bundleCount;

// Load data features computed from the above resources.
private double maxResourceUsage; // max of resource usages
Expand All @@ -72,6 +76,9 @@ public class BrokerLoadData {
private double weightedMaxEMA;
private long updatedAt;

@Setter
private long reportedAt;

public BrokerLoadData() {
cpu = new ResourceUsage();
memory = new ResourceUsage();
Expand All @@ -95,6 +102,8 @@ public BrokerLoadData() {
* broker-level message input rate in messages/s.
* @param msgRateOut
* broker-level message output rate in messages/s.
* @param bundleCount
* broker-level bundle counts.
* @param conf
* Service configuration to compute load data features.
*/
Expand All @@ -103,12 +112,14 @@ public void update(final SystemResourceUsage usage,
double msgThroughputOut,
double msgRateIn,
double msgRateOut,
int bundleCount,
ServiceConfiguration conf) {
updateSystemResourceUsage(usage.cpu, usage.memory, usage.directMemory, usage.bandwidthIn, usage.bandwidthOut);
this.msgThroughputIn = msgThroughputIn;
this.msgThroughputOut = msgThroughputOut;
this.msgRateIn = msgRateIn;
this.msgRateOut = msgRateOut;
this.bundleCount = bundleCount;
updateFeatures(conf);
updatedAt = System.currentTimeMillis();
}
Expand All @@ -125,9 +136,11 @@ public void update(final BrokerLoadData other) {
msgThroughputOut = other.msgThroughputOut;
msgRateIn = other.msgRateIn;
msgRateOut = other.msgRateOut;
bundleCount = other.bundleCount;
weightedMaxEMA = other.weightedMaxEMA;
maxResourceUsage = other.maxResourceUsage;
updatedAt = other.updatedAt;
reportedAt = other.reportedAt;
}

// Update resource usage given each individual usage.
Expand Down Expand Up @@ -177,7 +190,9 @@ public String toString(ServiceConfiguration conf) {
+ "cpuWeight= %f, memoryWeight= %f, directMemoryWeight= %f, "
+ "bandwithInResourceWeight= %f, bandwithOutResourceWeight= %f, "
+ "msgThroughputIn= %.2f, msgThroughputOut= %.2f, msgRateIn= %.2f, msgRateOut= %.2f, "
+ "maxResourceUsage= %.2f%%, weightedMaxEMA= %.2f%%, updatedAt= %d",
+ "bundleCount= %d, "
+ "maxResourceUsage= %.2f%%, weightedMaxEMA= %.2f%%, "
+ "updatedAt= %d, reportedAt= %d",

cpu.percentUsage(), memory.percentUsage(), directMemory.percentUsage(),
bandwidthIn.percentUsage(), bandwidthOut.percentUsage(),
Expand All @@ -187,7 +202,9 @@ public String toString(ServiceConfiguration conf) {
conf.getLoadBalancerBandwithInResourceWeight(),
conf.getLoadBalancerBandwithOutResourceWeight(),
msgThroughputIn, msgThroughputOut, msgRateIn, msgRateOut,
maxResourceUsage * 100, weightedMaxEMA * 100, updatedAt
bundleCount,
maxResourceUsage * 100, weightedMaxEMA * 100,
updatedAt, reportedAt
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,41 +18,30 @@
*/
package org.apache.pulsar.broker.loadbalance.extensions.data;

import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.ToString;
import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats;

/**
* Defines the information of top bundles load data.
*/
@Getter
@ToString
@EqualsAndHashCode
@NoArgsConstructor
public class TopBundlesLoadData {

private final List<BundleLoadData> topBundlesLoadData;
private final List<BundleLoadData> topBundlesLoadData = new ArrayList<>();

public record BundleLoadData(String bundleName, NamespaceBundleStats stats) {
public BundleLoadData {
Objects.requireNonNull(bundleName);
}
}

private TopBundlesLoadData(List<BundleLoadData> bundleStats, int topK) {
topBundlesLoadData = bundleStats
.stream()
.sorted((o1, o2) -> o2.stats().compareTo(o1.stats()))
.limit(topK)
.collect(Collectors.toList());
}

/**
* Give full bundle stats, and return the top K bundle stats.
*
* @param bundleStats full bundle stats.
* @param topK Top K bundles.
*/
public static TopBundlesLoadData of(List<BundleLoadData> bundleStats, int topK) {
return new TopBundlesLoadData(bundleStats, topK);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.loadbalance.extensions.models;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.ToString;
import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats;

/**
* Defines the information of top k highest-loaded bundles.
*/
@Getter
@ToString
@EqualsAndHashCode
@NoArgsConstructor
public class TopKBundles {

// temp array for sorting
private final List<Map.Entry<String, ? extends Comparable>> arr = new ArrayList<>();

private final TopBundlesLoadData loadData = new TopBundlesLoadData();

/**
* Update the topK bundles from the input bundleStats.
*
* @param bundleStats bundle stats.
* @param topk top k bundle stats to select.
*/
public void update(Map<String, NamespaceBundleStats> bundleStats, int topk) {
arr.clear();
for (var etr : bundleStats.entrySet()) {
if (etr.getKey().startsWith(NamespaceName.SYSTEM_NAMESPACE.toString())) {
continue;
}
arr.add(etr);
}
var topKBundlesLoadData = loadData.getTopBundlesLoadData();
topKBundlesLoadData.clear();
if (arr.isEmpty()) {
return;
}
topk = Math.min(topk, arr.size());
partitionSort(arr, topk);

for (int i = 0; i < topk; i++) {
var etr = arr.get(i);
topKBundlesLoadData.add(
new TopBundlesLoadData.BundleLoadData(etr.getKey(), (NamespaceBundleStats) etr.getValue()));
}
arr.clear();
}

static void partitionSort(List<Map.Entry<String, ? extends Comparable>> arr, int k) {
int start = 0;
int end = arr.size() - 1;
int target = k - 1;
while (start < end) {
int lo = start;
int hi = end;
int mid = lo;
var pivot = arr.get(hi).getValue();
while (mid <= hi) {
int cmp = pivot.compareTo(arr.get(mid).getValue());
if (cmp < 0) {
var tmp = arr.get(lo);
arr.set(lo++, arr.get(mid));
arr.set(mid++, tmp);
} else if (cmp > 0) {
var tmp = arr.get(mid);
arr.set(mid, arr.get(hi));
arr.set(hi--, tmp);
} else {
mid++;
}
}
if (lo <= target && target < mid) {
end = lo;
break;
}
if (target < lo) {
end = lo - 1;
} else {
start = mid;
}
}
Collections.sort(arr.subList(0, end), (a, b) -> b.getValue().compareTo(a.getValue()));
}
}
Loading