提交 f9993942 编写于 作者: S sschepens 提交者: Matteo Merli

Implement BrokerHostUsage using java (#88)

上级 5e9884c7
......@@ -212,10 +212,7 @@ loadBalancerReportUpdateThresholdPercentage=10
# maximum interval to update load report
loadBalancerReportUpdateMaxIntervalMinutes=15
# Path for the script used to retrieve system usage
loadBalancerHostUsageScriptPath=
# Frequency of sar report to collect
# Frequency of report to collect
loadBalancerHostUsageCheckIntervalMinutes=1
# Load shedding interval. Broker periodically checks whether some traffic should be offload from
......
......@@ -185,10 +185,7 @@ loadBalancerReportUpdateThresholdPercentage=10
# maximum interval to update load report
loadBalancerReportUpdateMaxIntervalMinutes=15
# Path for the script used to retrieve system usage
loadBalancerHostUsageScriptPath=
# Frequency of sar report to collect
# Frequency of report to collect
loadBalancerHostUsageCheckIntervalMinutes=1
# Load shedding interval. Broker periodically checks whether some traffic should be offload from
......
......@@ -186,10 +186,7 @@ public class ServiceConfiguration {
private int loadBalancerReportUpdateThresholdPercentage = 10;
// maximum interval to update load report
private int loadBalancerReportUpdateMaxIntervalMinutes = 15;
// Path for the script used to retrieve system usage
@FieldContext(required = false)
private String loadBalancerHostUsageScriptPath;
// Frequency of sar report to collect
// Frequency of report to collect
private int loadBalancerHostUsageCheckIntervalMinutes = 1;
// Load shedding interval. Broker periodically checks whether some traffic
// should be offload from
......@@ -715,14 +712,6 @@ public class ServiceConfiguration {
this.loadBalancerReportUpdateMaxIntervalMinutes = loadBalancerReportUpdateMaxIntervalMinutes;
}
public String getLoadBalancerHostUsageScriptPath() {
return loadBalancerHostUsageScriptPath;
}
public void setLoadBalancerHostUsageScriptPath(String loadBalancerHostUsageScriptPath) {
this.loadBalancerHostUsageScriptPath = loadBalancerHostUsageScriptPath;
}
public int getLoadBalancerHostUsageCheckIntervalMinutes() {
return loadBalancerHostUsageCheckIntervalMinutes;
}
......
......@@ -128,6 +128,7 @@ public class PulsarService implements AutoCloseable {
this.brokerServiceUrlTls = brokerUrlTls(config);
this.config = config;
this.shutdownService = new MessagingServiceShutdownHook(this);
loadManagerExecutor = Executors.newSingleThreadScheduledExecutor();
}
/**
......@@ -233,11 +234,11 @@ public class PulsarService implements AutoCloseable {
managedLedgerClientFactory = new ManagedLedgerClientFactory(config, getZkClient(),
getBookKeeperClientFactory());
this.brokerService = new BrokerService(this);
// Start load management service (even if load balancing is disabled)
this.loadManager = new SimpleLoadManagerImpl(this);
this.brokerService = new BrokerService(this);
this.startLoadManagementService();
// needs load management service
......@@ -400,11 +401,6 @@ public class PulsarService implements AutoCloseable {
this.loadManager.start();
if (config.isLoadBalancerEnabled()) {
if (loadManagerExecutor == null) {
loadManagerExecutor = Executors.newSingleThreadScheduledExecutor();
;
}
LOG.info("Starting load balancer");
if (this.loadReportTask == null) {
long loadReportMinInterval = SimpleLoadManagerImpl.LOAD_REPORT_UPDATE_MIMIMUM_INTERVAL;
......@@ -533,6 +529,10 @@ public class PulsarService implements AutoCloseable {
return executor;
}
public ScheduledExecutorService getLoadManagerExecutor() {
return loadManagerExecutor;
}
public OrderedSafeExecutor getOrderedExecutor() {
return orderedExecutor;
}
......
......@@ -16,80 +16,18 @@
package com.yahoo.pulsar.broker.loadbalance;
import java.io.IOException;
import java.io.StringWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.yahoo.pulsar.broker.PulsarService;
import com.yahoo.pulsar.utils.CmdUtility;
import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage;
/**
* Class that will return the broker host usage.
*
*
*/
public class BrokerHostUsage {
// The interval for host usage check command
private final int hostUsageCheckInterval;
// Path to the pulsar-broker-host-usage script
private final String usageScriptPath;
private static final Logger LOG = LoggerFactory.getLogger(BrokerHostUsage.class);
public BrokerHostUsage(PulsarService pulsar) {
this.usageScriptPath = pulsar.getConfiguration().getLoadBalancerHostUsageScriptPath();
this.hostUsageCheckInterval = pulsar.getConfiguration().getLoadBalancerHostUsageCheckIntervalMinutes();
}
public interface BrokerHostUsage {
/**
* Returns the host usage information in the following format -
*
* <pre>
* {
* "bandwidthIn" : {
* "usage" : "100",
* "limit" : "1000",
* },
* "bandwidthOut" : {
* "usage" : "659",
* "limit" : "1000",
* },
* "memory" : {
* "usage" : "16.0",
* "limit" : "16070",
* }
* "cpu-utilization" : {
* "usage" : "160.0"
* "limit" : "1600"
* }
* }
* </pre>
*
* @return Broker host usage in the json string format
*
* @throws IOException
*/
public String getBrokerHostUsage() throws IOException {
StringWriter writer = new StringWriter();
try {
/**
* Spawns a python process and runs the usage exporter script. The script return the machine information in
* the json format.
*/
int exitCode = CmdUtility.exec(writer, usageScriptPath, "--host-usage-check-interval",
Integer.toString(hostUsageCheckInterval));
if (exitCode != 0) {
LOG.warn("Process exited with non-zero exit code - [{}], stderr - [{}] ", exitCode, writer.toString());
throw new IOException(writer.toString());
}
} catch (IOException e) {
e.printStackTrace();
LOG.warn("Error running the usage script {}", e.getMessage());
throw e;
}
return writer.toString();
}
SystemResourceUsage getBrokerHostUsage();
}
/**
* 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.sun.management.OperatingSystemMXBean;
import com.yahoo.pulsar.broker.PulsarService;
import com.yahoo.pulsar.broker.loadbalance.BrokerHostUsage;
import com.yahoo.pulsar.common.policies.data.loadbalancer.ResourceUsage;
import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.concurrent.TimeUnit;
/**
* Class that will return the broker host usage.
*/
public class GenericBrokerHostUsageImpl implements BrokerHostUsage {
// The interval for host usage check command
private static final int CPU_CHECK_MILLIS = 1000;
private static final Logger LOG = LoggerFactory.getLogger(GenericBrokerHostUsageImpl.class);
private final int hostUsageCheckIntervalMin;
private long lastCollection;
private double totalCpuLimit;
private double cpuUsageSum = 0d;
private int cpuUsageCount = 0;
private OperatingSystemMXBean systemBean;
private SystemResourceUsage usage;
public GenericBrokerHostUsageImpl(PulsarService pulsar) {
this.hostUsageCheckIntervalMin = pulsar.getConfiguration().getLoadBalancerHostUsageCheckIntervalMinutes();
this.systemBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean();
this.lastCollection = 0L;
this.usage = new SystemResourceUsage();
this.totalCpuLimit = getTotalCpuLimit();
pulsar.getLoadManagerExecutor().scheduleAtFixedRate(this::checkCpuLoad, 0, CPU_CHECK_MILLIS, TimeUnit.MILLISECONDS);
pulsar.getLoadManagerExecutor().scheduleAtFixedRate(this::calculateBrokerHostUsage, 0, hostUsageCheckIntervalMin, TimeUnit.MINUTES);
}
@Override
public SystemResourceUsage getBrokerHostUsage() {
return usage;
}
private void checkCpuLoad() {
cpuUsageSum += systemBean.getSystemCpuLoad();
cpuUsageCount++;
}
private void calculateBrokerHostUsage() {
SystemResourceUsage usage = new SystemResourceUsage();
usage.setCpu(getCpuUsage());
usage.setMemory(getMemUsage());
this.usage = usage;
}
private double getTotalCpuLimit() {
return (double) (100 * Runtime.getRuntime().availableProcessors());
}
private double getTotalCpuUsage() {
double cpuUsage = cpuUsageSum / cpuUsageCount;
cpuUsageSum = 0d;
cpuUsageCount = 0;
return cpuUsage;
}
private ResourceUsage getCpuUsage() {
if (cpuUsageCount == 0) {
return new ResourceUsage(0, totalCpuLimit);
}
return new ResourceUsage(getTotalCpuUsage() * totalCpuLimit, totalCpuLimit);
}
private ResourceUsage getMemUsage() {
double total = ((double) systemBean.getTotalPhysicalMemorySize()) / (1024 * 1024);
double free = ((double) systemBean.getFreePhysicalMemorySize()) / (1024 * 1024);
return new ResourceUsage(total - free, total);
}
}
/**
* 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.sun.management.OperatingSystemMXBean;
import com.yahoo.pulsar.broker.PulsarService;
import com.yahoo.pulsar.broker.loadbalance.BrokerHostUsage;
import com.yahoo.pulsar.common.policies.data.loadbalancer.ResourceUsage;
import com.yahoo.pulsar.common.policies.data.loadbalancer.SystemResourceUsage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
/**
* Class that will return the broker host usage.
*
*
*/
public class LinuxBrokerHostUsageImpl implements BrokerHostUsage {
// The interval for host usage check command
private final int hostUsageCheckInterval;
private long lastCollection;
private double lastTotalNicUsageTx;
private double lastTotalNicUsageRx;
private CpuStat lastCpuStat;
private OperatingSystemMXBean systemBean;
private SystemResourceUsage usage;
private static final Logger LOG = LoggerFactory.getLogger(LinuxBrokerHostUsageImpl.class);
public LinuxBrokerHostUsageImpl(PulsarService pulsar) {
this.hostUsageCheckInterval = pulsar.getConfiguration().getLoadBalancerHostUsageCheckIntervalMinutes();
this.systemBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean();
this.lastCollection = 0L;
this.usage = new SystemResourceUsage();
pulsar.getLoadManagerExecutor().scheduleAtFixedRate(this::calculateBrokerHostUsage, 0, hostUsageCheckInterval, TimeUnit.SECONDS);
}
@Override
public SystemResourceUsage getBrokerHostUsage() {
return usage;
}
private void calculateBrokerHostUsage() {
List<String> nics = getNics();
double totalNicLimit = getTotalNicLimitKbps(nics);
double totalNicUsageTx = getTotalNicUsageTxKb(nics);
double totalNicUsageRx = getTotalNicUsageRxKb(nics);
double totalCpuLimit = getTotalCpuLimit();
CpuStat cpuStat = getTotalCpuUsage();
SystemResourceUsage usage = new SystemResourceUsage();
long now = System.currentTimeMillis();
if (lastCollection == 0L) {
usage.setMemory(getMemUsage());
usage.setBandwidthIn(new ResourceUsage(0d, totalNicLimit));
usage.setBandwidthOut(new ResourceUsage(0d, totalNicLimit));
usage.setCpu(new ResourceUsage(0d, totalCpuLimit));
} else {
double elapsedSeconds = (now - lastCollection) / 1000d;
double nicUsageTx = (totalNicUsageTx - lastTotalNicUsageTx) / elapsedSeconds;
double nicUsageRx = (totalNicUsageRx - lastTotalNicUsageRx) / elapsedSeconds;
if (cpuStat != null && lastCpuStat != null) {
// we need two non null stats to get a usage report
long cpuTimeDiff = cpuStat.getTotalTime() - lastCpuStat.getTotalTime();
long cpuUsageDiff = cpuStat.getUsage() - lastCpuStat.getUsage();
double cpuUsage = ((double) cpuUsageDiff / (double) cpuTimeDiff) * totalCpuLimit;
usage.setCpu(new ResourceUsage(cpuUsage, totalCpuLimit));
}
usage.setMemory(getMemUsage());
usage.setBandwidthIn(new ResourceUsage(nicUsageRx, totalNicLimit));
usage.setBandwidthOut(new ResourceUsage(nicUsageTx, totalNicLimit));
}
lastTotalNicUsageTx = totalNicUsageTx;
lastTotalNicUsageRx = totalNicUsageRx;
lastCpuStat = cpuStat;
lastCollection = System.currentTimeMillis();
this.usage = usage;
}
private double getTotalCpuLimit() {
return (double) (100 * Runtime.getRuntime().availableProcessors());
}
/**
* Reads first line of /proc/stat to get total cpu usage.
* <pre>
* cpu user nice system idle iowait irq softirq steal guest guest_nice
* cpu 317808 128 58637 2503692 7634 0 13472 0 0 0
* </pre>
* 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 {
String[] words = Files.lines(Paths.get("/proc/stat"))
.findFirst()
.get().split("\\s+");
long total = Arrays.stream(words)
.filter(s -> !s.contains("cpu"))
.mapToLong(Long::parseLong)
.sum();
long idle = Long.parseLong(words[4]);
return new CpuStat(total, total - idle);
} catch (IOException e) {
LOG.error("Failed to read CPU usage from /proc/stat", e);
return null;
}
}
private ResourceUsage getMemUsage() {
double total = ((double) systemBean.getTotalPhysicalMemorySize()) / (1024 * 1024);
double free = ((double) systemBean.getFreePhysicalMemorySize()) / (1024 * 1024);
return new ResourceUsage(total - free, total);
}
private List<String> getNics() {
try {
return Files.list(Paths.get("/sys/class/net/"))
.filter(this::isPhysicalNic)
.map(path -> path.getFileName().toString())
.collect(Collectors.toList());
} catch (IOException e) {
LOG.error("Failed to find NICs", e);
return Collections.emptyList();
}
}
private boolean isPhysicalNic(Path path) {
try {
if (!Files.readSymbolicLink(path).toString().contains("/virtual/")) {
try {
Files.readAllBytes(path.resolve("speed"));
return true;
} catch (Exception e) {
// wireless nics don't report speed, ignore them.
return false;
}
}
return false;
} catch (IOException e) {
LOG.error("Failed to read link target for NIC " + path, e);
return false;
}
}
private Path getNicSpeedPath(String nic) {
return Paths.get(String.format("/sys/class/net/%s/speed", nic));
}
private double getTotalNicLimitKbps(List<String> nics) {
// Nic speed is in Mbits/s, return kbits/s
return nics.stream().mapToDouble(s -> {
try {
return Double.parseDouble(new String(Files.readAllBytes(getNicSpeedPath(s))));
} catch (IOException e) {
LOG.error("Failed to read speed for nic " + s, e);
return 0d;
}
}).sum() * 1024;
}
private Path getNicTxPath(String nic) {
return Paths.get(String.format("/sys/class/net/%s/statistics/tx_bytes", nic));
}
private Path getNicRxPath(String nic) {
return Paths.get(String.format("/sys/class/net/%s/statistics/rx_bytes", nic));
}
private double getTotalNicUsageRxKb(List<String> nics) {
return nics.stream().mapToDouble(s -> {
try {
return Double.parseDouble(new String(Files.readAllBytes(getNicRxPath(s))));
} catch (IOException e) {
LOG.error("Failed to read rx_bytes for NIC " + s, e);
return 0d;
}
}).sum() * 8 / 1024;
}
private double getTotalNicUsageTxKb(List<String> nics) {
return nics.stream().mapToDouble(s -> {
try {
return Double.parseDouble(new String(Files.readAllBytes(getNicTxPath(s))));
} catch (IOException e) {
LOG.error("Failed to read tx_bytes for NIC " + s, e);
return 0d;
}
}).sum() * 8 / 1024;
}
private class CpuStat {
private long totalTime;
private long usage;
CpuStat(long totalTime, long usage) {
this.totalTime = totalTime;
this.usage = usage;
}
long getTotalTime() {
return totalTime;
}
long getUsage() {
return usage;
}
}
}
......@@ -168,7 +168,6 @@ public class PulsarBrokerStarterTest {
printWriter.println("managedLedgerDefaultAckQuorum=1");
printWriter.println("loadBalancerEnabled=false");
printWriter.println("loadBalancerHostUsageScriptPath=/usr/bin/my_pulsar-broker-host-usage");
printWriter.println("loadBalancerHostUsageCheckIntervalMinutes=4");
printWriter.println("loadBalancerReportUpdateThresholdPercentage=15");
printWriter.println("loadBalancerReportUpdateMaxIntervalMinutes=20");
......@@ -186,7 +185,6 @@ public class PulsarBrokerStarterTest {
Assert.assertTrue(ServiceConfiguration.class.isInstance(returnValue));
ServiceConfiguration serviceConfig = (ServiceConfiguration) returnValue;
Assert.assertEquals(serviceConfig.isLoadBalancerEnabled(), false);
Assert.assertEquals(serviceConfig.getLoadBalancerHostUsageScriptPath(), "/usr/bin/my_pulsar-broker-host-usage");
Assert.assertEquals(serviceConfig.getLoadBalancerHostUsageCheckIntervalMinutes(), 4);
Assert.assertEquals(serviceConfig.getLoadBalancerReportUpdateThresholdPercentage(), 15);
Assert.assertEquals(serviceConfig.getLoadBalancerReportUpdateMaxIntervalMinutes(), 20);
......
......@@ -211,9 +211,11 @@ public class LoadBalancerTest {
printSortedRanking(sortedRanking);
// all brokers have same rank to it would be 0 --> set-of-all-the-brokers
assertEquals(sortedRanking.get().size(), 1);
assertTrue(sortedRanking.get().get(0L) != null);
assertEquals(sortedRanking.get().get(0L).size(), BROKER_COUNT);
int brokerCount = 0;
for (Map.Entry<Long, Set<ResourceUnit>> entry : sortedRanking.get().entrySet()) {
brokerCount += entry.getValue().size();
}
assertEquals(brokerCount, BROKER_COUNT);
DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns/test-topic");
ResourceUnit found = pulsarServices[i].getLoadManager()
.getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(fqdn));
......
......@@ -27,6 +27,7 @@ import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import org.apache.commons.lang3.SystemUtils;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.URL;
......@@ -41,6 +42,7 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import com.yahoo.pulsar.broker.loadbalance.impl.*;
import org.apache.bookkeeper.test.PortManager;
import org.apache.bookkeeper.util.ZkUtils;
import org.apache.zookeeper.CreateMode;
......@@ -57,12 +59,6 @@ import com.google.common.collect.Sets;
import com.yahoo.pulsar.broker.PulsarService;
import com.yahoo.pulsar.broker.ServiceConfiguration;
import com.yahoo.pulsar.broker.admin.AdminResource;
import com.yahoo.pulsar.broker.loadbalance.impl.PulsarLoadReportImpl;
import com.yahoo.pulsar.broker.loadbalance.impl.PulsarResourceDescription;
import com.yahoo.pulsar.broker.loadbalance.impl.ResourceAvailabilityRanker;
import com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadCalculatorImpl;
import com.yahoo.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl;
import com.yahoo.pulsar.broker.loadbalance.impl.SimpleResourceUnit;
import com.yahoo.pulsar.client.admin.BrokerStats;
import com.yahoo.pulsar.client.admin.PulsarAdmin;
import com.yahoo.pulsar.client.api.Authentication;
......@@ -443,14 +439,14 @@ public class SimpleLoadManagerImplTest {
@Test
public void testBrokerHostUsage() {
when(pulsar1.getConfiguration().getLoadBalancerHostUsageScriptPath()).thenReturn("usageScript");
BrokerHostUsage brokerUsage = new BrokerHostUsage(pulsar1);
try {
brokerUsage.getBrokerHostUsage();
fail();
} catch (IOException e) {
// Ok
BrokerHostUsage brokerUsage;
if (SystemUtils.IS_OS_LINUX) {
brokerUsage = new LinuxBrokerHostUsageImpl(pulsar1);
} else {
brokerUsage = new GenericBrokerHostUsageImpl(pulsar1);
}
brokerUsage.getBrokerHostUsage();
// Ok
}
@Test
......
......@@ -52,7 +52,6 @@ managedLedgerCursorRolloverTimeInSeconds=14400
loadBalancerEnabled=false
loadBalancerReportUpdateThresholdPercentage=10
loadBalancerReportUpdateMaxIntervalMinutes=15
loadBalancerHostUsageScriptPath=/usr/bin/pulsar-broker-host-usage
loadBalancerHostUsageCheckIntervalMinutes=1
loadBalancerSheddingIntervalMinutes=30
loadBalancerSheddingGracePeriodMinutes=30
......
#!/bin/bash
# response to test the load-report API : BrokerHostUsage execs a script that returns system resource usage;
# Since the original script is system dependent and also requires /home/y location, this script is the dummy one
# returning a fake/mock response, so we can test the load-report API end-to-end including the path executing the script
echo "{\"bandwidthIn\":{\"usage\": 0,\"limit\": 0},\"bandwidthOut\":{\"usage\": 0,\"limit\": 0},\"memory\":{\"usage\":3240,\"limit\": 12829},\"cpu\": {\"usage\": 10.0,\"limit\": 16}, \"threads\":{\"usage\":10,\"limit\":100}}"
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册