diff --git a/config/kraft/broker.properties b/config/kraft/broker.properties index bb9cfcec15..7e102d4e57 100644 --- a/config/kraft/broker.properties +++ b/config/kraft/broker.properties @@ -182,9 +182,3 @@ s3.wal.path=/tmp/kraft-broker-logs/s3wal ############################# Settings for Auto Balancer ############################# # The metric reporter to collect and report metrics for Auto Balancer metric.reporters=kafka.autobalancer.metricsreporter.AutoBalancerMetricsReporter - -# The network inbound bandwidth in bytes/s, default 100MB/s. Used in NetworkInCapacityGoal and calculation of inbound bandwidth utilization -# autobalancer.reporter.network.in.capacity=104857600 - -# The network outbound bandwidth in bytes/s, default 100MB/s. Used in NetworkOutCapacityGoal and calculation of outbound bandwidth utilization -# autobalancer.reporter.network.out.capacity=104857600 diff --git a/config/kraft/server.properties b/config/kraft/server.properties index efe1ad0a82..506e4321e5 100644 --- a/config/kraft/server.properties +++ b/config/kraft/server.properties @@ -189,12 +189,6 @@ s3.wal.path=/tmp/kraft-combined-logs/s3wal # The metric reporter to collect and report metrics for Auto Balancer metric.reporters=kafka.autobalancer.metricsreporter.AutoBalancerMetricsReporter -# The network inbound bandwidth in bytes/s, default 100MB/s. Used in NetworkInCapacityGoal and calculation of inbound bandwidth utilization -# autobalancer.reporter.network.in.capacity=104857600 - -# The network outbound bandwidth in bytes/s, default 100MB/s. Used in NetworkOutCapacityGoal and calculation of outbound bandwidth utilization -# autobalancer.reporter.network.out.capacity=104857600 - ############################# Settings of Controller for Auto Balancer ############################# # Whether to enabled Auto Balancer in controller, default false # autobalancer.controller.enable=false diff --git a/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java b/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java index 5d878fdd3a..e2ed977ff0 100644 --- a/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java +++ b/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java @@ -54,7 +54,7 @@ import java.util.List; import java.util.Set; -public class AutoBalancerManager { +public class AutoBalancerManager implements AutoBalancerService { private final Logger logger; private final LoadRetriever loadRetriever; private final AnomalyDetector anomalyDetector; @@ -65,7 +65,7 @@ public AutoBalancerManager(Time time, KafkaConfig kafkaConfig, QuorumController LogContext logContext = new LogContext(String.format("[AutoBalancerManager id=%d] ", quorumController.nodeId())); logger = logContext.logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); AutoBalancerControllerConfig config = new AutoBalancerControllerConfig(kafkaConfig.props(), false); - RecordClusterModel clusterModel = new RecordClusterModel(new LogContext(String.format("[ClusterModel id=%d] ", quorumController.nodeId()))); + RecordClusterModel clusterModel = buildClusterModel(quorumController.nodeId()); this.loadRetriever = new LoadRetriever(config, quorumController, clusterModel, new LogContext(String.format("[LoadRetriever id=%d] ", quorumController.nodeId()))); ControllerActionExecutorService actionExecutorService = new ControllerActionExecutorService(config, quorumController, @@ -77,7 +77,6 @@ public AutoBalancerManager(Time time, KafkaConfig kafkaConfig, QuorumController .detectIntervalMs(config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS)) .maxTolerateMetricsDelayMs(config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS)) .coolDownIntervalPerActionMs(config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS)) - .aggregateBrokerLoad(config.getBoolean(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION)) .clusterModel(clusterModel) .executor(actionExecutorService) .addGoals(config.getConfiguredInstances(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, Goal.class)) @@ -95,19 +94,30 @@ public AutoBalancerManager(Time time, KafkaConfig kafkaConfig, QuorumController raftClient.register(new AutoBalancerListener(registry, this.loadRetriever, this.anomalyDetector)); } + @Override public void start() { loadRetriever.start(); anomalyDetector.start(); logger.info("Started"); } - public void shutdown() throws InterruptedException { - anomalyDetector.shutdown(); - loadRetriever.shutdown(); - queue.close(); + @Override + public void shutdown() { + try { + anomalyDetector.shutdown(); + loadRetriever.shutdown(); + queue.close(); + } catch (Exception e) { + logger.error("Exception in shutdown", e); + } + logger.info("Shutdown completed"); } + public RecordClusterModel buildClusterModel(int nodeId) { + return new RecordClusterModel(new LogContext(String.format("[ClusterModel id=%d] ", nodeId))); + } + private Set parseExcludedBrokers(AutoBalancerControllerConfig config) { Set excludedBrokers = new HashSet<>(); for (String brokerId : config.getList(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS)) { diff --git a/core/src/main/java/kafka/autobalancer/goals/GoalConstants.java b/core/src/main/java/kafka/autobalancer/AutoBalancerService.java similarity index 80% rename from core/src/main/java/kafka/autobalancer/goals/GoalConstants.java rename to core/src/main/java/kafka/autobalancer/AutoBalancerService.java index f62397281e..1c3ff2975c 100644 --- a/core/src/main/java/kafka/autobalancer/goals/GoalConstants.java +++ b/core/src/main/java/kafka/autobalancer/AutoBalancerService.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package kafka.autobalancer.goals; +package kafka.autobalancer; -public class GoalConstants { - public static final int NETWORK_DISTRIBUTION_GOAL_PRIORITY = 8; - public static final int NETWORK_CAPACITY_GOAL_PRIORITY = 10; +public interface AutoBalancerService { + void start(); + void shutdown(); } diff --git a/core/src/main/java/kafka/autobalancer/LoadRetriever.java b/core/src/main/java/kafka/autobalancer/LoadRetriever.java index 30ba401e62..2495bddc14 100644 --- a/core/src/main/java/kafka/autobalancer/LoadRetriever.java +++ b/core/src/main/java/kafka/autobalancer/LoadRetriever.java @@ -414,13 +414,13 @@ private void updateClusterModel(AutoBalancerMetrics metrics) { switch (metrics.metricClassId()) { case BROKER_METRIC: BrokerMetrics brokerMetrics = (BrokerMetrics) metrics; - clusterModel.updateBrokerMetrics(brokerMetrics.brokerId(), brokerMetrics.getMetricTypeValueMap(), brokerMetrics.time()); + clusterModel.updateBrokerMetrics(brokerMetrics.brokerId(), brokerMetrics.getMetricValueMap(), brokerMetrics.time()); break; case PARTITION_METRIC: TopicPartitionMetrics partitionMetrics = (TopicPartitionMetrics) metrics; clusterModel.updateTopicPartitionMetrics(partitionMetrics.brokerId(), new TopicPartition(partitionMetrics.topic(), partitionMetrics.partition()), - partitionMetrics.getMetricTypeValueMap(), partitionMetrics.time()); + partitionMetrics.getMetricValueMap(), partitionMetrics.time()); break; default: logger.error("Not supported metrics version {}", metrics.metricClassId()); diff --git a/core/src/main/java/kafka/autobalancer/common/Action.java b/core/src/main/java/kafka/autobalancer/common/Action.java index 01860089f9..3936d3dc9a 100644 --- a/core/src/main/java/kafka/autobalancer/common/Action.java +++ b/core/src/main/java/kafka/autobalancer/common/Action.java @@ -17,6 +17,7 @@ package kafka.autobalancer.common; +import java.util.Objects; import org.apache.kafka.common.TopicPartition; public class Action { @@ -78,4 +79,18 @@ public String prettyString() { } return toString(); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Action action = (Action) o; + return srcBrokerId == action.srcBrokerId && destBrokerId == action.destBrokerId + && Objects.equals(srcTp, action.srcTp) && Objects.equals(destTp, action.destTp) && type == action.type; + } + + @Override + public int hashCode() { + return Objects.hash(srcTp, destTp, srcBrokerId, destBrokerId, type); + } } diff --git a/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java b/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java index 6eceb7c982..6f6e92134a 100644 --- a/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java +++ b/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java @@ -17,14 +17,14 @@ package kafka.autobalancer.common; +import com.automq.stream.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; public class AutoBalancerThreadFactory implements ThreadFactory { - private static final Logger LOGGER = LoggerFactory.getLogger(AutoBalancerThreadFactory.class); + private static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); private final String name; private final boolean daemon; private final AtomicInteger id = new AtomicInteger(0); diff --git a/core/src/main/java/kafka/autobalancer/common/RawMetricType.java b/core/src/main/java/kafka/autobalancer/common/RawMetricType.java index e927357cb8..c1633ad43a 100644 --- a/core/src/main/java/kafka/autobalancer/common/RawMetricType.java +++ b/core/src/main/java/kafka/autobalancer/common/RawMetricType.java @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -41,9 +40,6 @@ public enum RawMetricType { TOPIC_PARTITION_BYTES_OUT(MetricScope.PARTITION, (byte) 5, (byte) 0), PARTITION_SIZE(MetricScope.PARTITION, (byte) 6, (byte) 0), BROKER_CPU_UTIL(MetricScope.BROKER, (byte) 7, (byte) 0); - - private static final List CACHED_VALUES = List.of(RawMetricType.values()); - private static final SortedMap> BROKER_METRIC_TYPES_DIFF_BY_VERSION = buildBrokerMetricTypesDiffByVersion(); private static final List BROKER_METRIC_TYPES = Collections.unmodifiableList(buildMetricTypeList(MetricScope.BROKER)); private static final List PARTITION_METRIC_TYPES = Collections.unmodifiableList(buildMetricTypeList(MetricScope.PARTITION)); private final byte id; @@ -60,18 +56,6 @@ public enum RawMetricType { this.supportedVersionSince = supportedVersionSince; } - public static List allMetricTypes() { - return Collections.unmodifiableList(CACHED_VALUES); - } - - public static Map> brokerMetricTypesDiffByVersion() { - return BROKER_METRIC_TYPES_DIFF_BY_VERSION; - } - - public static Set brokerMetricTypesDiffForVersion(byte version) { - return BROKER_METRIC_TYPES_DIFF_BY_VERSION.get(version); - } - public static List partitionMetricTypes() { return PARTITION_METRIC_TYPES; } diff --git a/core/src/main/java/kafka/autobalancer/common/Resource.java b/core/src/main/java/kafka/autobalancer/common/Resource.java index e5e9ea2636..fa63bd87fb 100644 --- a/core/src/main/java/kafka/autobalancer/common/Resource.java +++ b/core/src/main/java/kafka/autobalancer/common/Resource.java @@ -28,7 +28,7 @@ public enum Resource { NW_OUT("NWOut", 2, 10), UNKNOWN("UNKNOWN", 999, 0); - public static final Double IGNORED_CAPACITY_VALUE = -1.0; + public static final Double IGNORED_VALUE = -1.0; // EPSILON_PERCENT defines the acceptable nuance when comparing the utilization of the resource. // This nuance is generated due to precision loss when summing up float type utilization value. // In stress test we find that for cluster of around 800,000 replicas, the summed up nuance can be @@ -58,7 +58,7 @@ public static Resource of(int id) { public String resourceString(double value) { String valueStr = ""; - if (value == IGNORED_CAPACITY_VALUE) { + if (value == IGNORED_VALUE) { valueStr = "ignored"; } else { switch (this) { diff --git a/core/src/main/java/kafka/autobalancer/common/normalizer/LinearNormalizer.java b/core/src/main/java/kafka/autobalancer/common/normalizer/LinearNormalizer.java new file mode 100644 index 0000000000..b93c54c9de --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/normalizer/LinearNormalizer.java @@ -0,0 +1,36 @@ +/* + * 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 kafka.autobalancer.common.normalizer; + +/** + * Linear normalizer that normalize the value to [0, 1] + */ +public class LinearNormalizer implements Normalizer { + public final double min; + public final double max; + + public LinearNormalizer(double min, double max) { + this.min = min; + this.max = max; + } + + @Override + public double normalize(double value) { + return (value - min) / (max - min); + } +} diff --git a/core/src/main/java/kafka/autobalancer/common/normalizer/Normalizer.java b/core/src/main/java/kafka/autobalancer/common/normalizer/Normalizer.java new file mode 100644 index 0000000000..87981b5f31 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/normalizer/Normalizer.java @@ -0,0 +1,33 @@ +/* + * 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 kafka.autobalancer.common.normalizer; + +public interface Normalizer { + + /** + * Normalize the value to [0, 1] + * + * @param value the value to normalize + * @return the normalized value + */ + double normalize(double value); + default double normalize(double value, boolean reverse) { + double normalizedValue = normalize(value); + return reverse ? (1 - normalizedValue) : normalizedValue; + } +} diff --git a/core/src/main/java/kafka/autobalancer/common/normalizer/StepNormalizer.java b/core/src/main/java/kafka/autobalancer/common/normalizer/StepNormalizer.java new file mode 100644 index 0000000000..79815539a2 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/normalizer/StepNormalizer.java @@ -0,0 +1,50 @@ +/* + * 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 kafka.autobalancer.common.normalizer; + +/** + * Step normalizer that normalize the value to [0, 1], when value is less than stepVar, it will be normalized with + * LinearNormalizer, otherwise it will be normalized with a logarithmic function which approaches 1 while the value + * approaches infinity. + */ +public class StepNormalizer implements Normalizer { + private final double stepValue; + private final double stepVar; + private final Normalizer linearNormalizer; + + public StepNormalizer(double min, double stepVar, double stepValue) { + if (stepValue < 0 || stepValue > 1) { + throw new IllegalArgumentException("Step value must be in [0, 1]"); + } + this.stepVar = stepVar; + this.stepValue = stepValue; + this.linearNormalizer = new LinearNormalizer(min, this.stepVar); + } + + @Override + public double normalize(double value) { + if (value <= stepVar) { + return stepValue * linearNormalizer.normalize(value); + } + return stepValue + delta(value); + } + + private double delta(double value) { + return (1 - this.stepValue) * (1 - 1 / (Math.log(value) / Math.log(stepVar))); + } +} diff --git a/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java b/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java index 5445ccc71c..984358ead5 100644 --- a/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java +++ b/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java @@ -17,10 +17,8 @@ package kafka.autobalancer.config; -import kafka.autobalancer.goals.NetworkInCapacityGoal; -import kafka.autobalancer.goals.NetworkInDistributionGoal; -import kafka.autobalancer.goals.NetworkOutCapacityGoal; -import kafka.autobalancer.goals.NetworkOutDistributionGoal; +import kafka.autobalancer.goals.NetworkInUsageDistributionGoal; +import kafka.autobalancer.goals.NetworkOutUsageDistributionGoal; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.ConfigDef; @@ -38,12 +36,10 @@ public class AutoBalancerControllerConfig extends AutoBalancerConfig { public static final String AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS = PREFIX + "metrics.delay.ms"; public static final String AUTO_BALANCER_CONTROLLER_GOALS = PREFIX + "goals"; public static final String AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS = PREFIX + "anomaly.detect.interval.ms"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD = PREFIX + "network.in.distribution.detect.threshold"; + public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD = PREFIX + "network.in.usage.distribution.detect.threshold"; public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION = PREFIX + "network.in.distribution.detect.avg.deviation"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD = PREFIX + "network.out.distribution.detect.threshold"; + public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD = PREFIX + "network.out.usage.distribution.detect.threshold"; public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION = PREFIX + "network.out.distribution.detect.avg.deviation"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_UTILIZATION_THRESHOLD = PREFIX + "network.in.utilization.threshold"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_UTILIZATION_THRESHOLD = PREFIX + "network.out.utilization.threshold"; public static final String AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS = PREFIX + "execution.interval.ms"; public static final String AUTO_BALANCER_CONTROLLER_EXECUTION_STEPS = PREFIX + "execution.steps"; public static final String AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION = PREFIX + "load.aggregation"; @@ -56,17 +52,13 @@ public class AutoBalancerControllerConfig extends AutoBalancerConfig { public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_RETRY_BACKOFF_MS = 1000; public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS = Duration.ofMinutes(1).toMillis(); public static final String DEFAULT_AUTO_BALANCER_CONTROLLER_GOALS = new StringJoiner(",") - .add(NetworkInCapacityGoal.class.getName()) - .add(NetworkOutCapacityGoal.class.getName()) - .add(NetworkInDistributionGoal.class.getName()) - .add(NetworkOutDistributionGoal.class.getName()).toString(); + .add(NetworkInUsageDistributionGoal.class.getName()) + .add(NetworkOutUsageDistributionGoal.class.getName()).toString(); public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS = 60000; - public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD = 0.2; + public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD = 1024 * 1024; public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION = 0.2; - public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD = 0.2; + public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD = 1024 * 1024; public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION = 0.2; - public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_THRESHOLD = 0.8; - public static final double DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_THRESHOLD = 0.8; public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS = 1000; public static final int DEFAULT_AUTO_BALANCER_CONTROLLER_EXECUTION_STEPS = 60; public static final boolean DEFAULT_AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION = false; @@ -76,17 +68,14 @@ public class AutoBalancerControllerConfig extends AutoBalancerConfig { public static final String AUTO_BALANCER_CONTROLLER_ENABLE_DOC = "Whether to enable auto balancer"; public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT_DOC = "The maximum time to block for one poll request in millisecond"; public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging."; - public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX_DOC = CommonClientConfigs.GROUP_ID_DOC; public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; public static final String AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS_DOC = "The maximum delayed time to consider a metrics valid"; public static final String AUTO_BALANCER_CONTROLLER_GOALS_DOC = "The goals to be detect in anomaly detector"; public static final String AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS_DOC = "Time interval between anomaly detections in milliseconds"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD_DOC = "The network input bandwidth usage detect threshold"; + public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD_DOC = "The network input bandwidth usage detect threshold in bytes"; public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION_DOC = "The acceptable range of deviation for average network input bandwidth usage"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD_DOC = "The network output bandwidth usage detect threshold"; + public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD_DOC = "The network output bandwidth usage detect threshold in bytes"; public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION_DOC = "The acceptable range of deviation for average network output bandwidth usage"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_THRESHOLD_DOC = "The maximum network input bandwidth usage of broker before trigger load balance"; - public static final String AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_THRESHOLD_DOC = PREFIX + "network.out.usage.threshold"; public static final String AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS_DOC = "Time interval between reassignments per broker in milliseconds"; public static final String AUTO_BALANCER_CONTROLLER_EXECUTION_STEPS_DOC = "The max number of reassignments per broker in one execution"; public static final String AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION_DOC = "Use aggregation of partition load as broker load, instead of using reported broker metrics directly"; @@ -112,24 +101,18 @@ public class AutoBalancerControllerConfig extends AutoBalancerConfig { .define(AUTO_BALANCER_CONTROLLER_GOALS, ConfigDef.Type.LIST, DEFAULT_AUTO_BALANCER_CONTROLLER_GOALS, ConfigDef.Importance.HIGH, AUTO_BALANCER_CONTROLLER_GOALS_DOC) - .define(AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Importance.HIGH, - AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD_DOC) + .define(AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD_DOC) .define(AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION, ConfigDef.Type.DOUBLE, DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION, ConfigDef.Importance.HIGH, AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION_DOC) - .define(AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Importance.HIGH, - AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD_DOC) + .define(AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD_DOC) .define(AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION, ConfigDef.Type.DOUBLE, DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION, ConfigDef.Importance.HIGH, AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION_DOC) - .define(AUTO_BALANCER_CONTROLLER_NETWORK_IN_UTILIZATION_THRESHOLD, ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_THRESHOLD, ConfigDef.Importance.HIGH, - AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_THRESHOLD_DOC) - .define(AUTO_BALANCER_CONTROLLER_NETWORK_OUT_UTILIZATION_THRESHOLD, ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_THRESHOLD, ConfigDef.Importance.HIGH, - AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_THRESHOLD_DOC) .define(AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS, ConfigDef.Type.LONG, DEFAULT_AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS, ConfigDef.Importance.HIGH, AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS_DOC) diff --git a/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java b/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java index acb19428bb..29ca84f5f0 100644 --- a/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java +++ b/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java @@ -34,8 +34,6 @@ public class AutoBalancerMetricsReporterConfig extends AutoBalancerConfig { private static final Set CONFIGS = new HashSet<>(); /* Configurations */ private static final String PREFIX = "autobalancer.reporter."; - public static final String AUTO_BALANCER_BROKER_NW_IN_CAPACITY = PREFIX + "network.in.capacity"; - public static final String AUTO_BALANCER_BROKER_NW_OUT_CAPACITY = PREFIX + "network.out.capacity"; public static final String AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS_CONFIG = PREFIX + "topic.auto.create.timeout.ms"; public static final String AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES_CONFIG = PREFIX + "topic.auto.create.retries"; public static final String AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES_CONFIG = PREFIX + "producer.create.retries"; @@ -43,10 +41,7 @@ public class AutoBalancerMetricsReporterConfig extends AutoBalancerConfig { public static final String AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID = PREFIX + "producer.client.id"; public static final String AUTO_BALANCER_METRICS_REPORTER_LINGER_MS_CONFIG = PREFIX + "producer.linger.ms"; public static final String AUTO_BALANCER_METRICS_REPORTER_BATCH_SIZE_CONFIG = PREFIX + "producer.batch.size"; - public static final String AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE_CONFIG = PREFIX + "kubernetes.mode"; /* Default values */ - public static final double DEFAULT_AUTO_BALANCER_BROKER_NW_IN_CAPACITY = 100 * 1024 * 1024; // 100MB/s - public static final double DEFAULT_AUTO_BALANCER_BROKER_NW_OUT_CAPACITY = 100 * 1024 * 1024; // 100MB/s public static final String DEFAULT_AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID = "AutoBalancerMetricsReporterProducer"; public static final long DEFAULT_AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10); public static final Integer DEFAULT_AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES = 5; @@ -54,11 +49,8 @@ public class AutoBalancerMetricsReporterConfig extends AutoBalancerConfig { public static final long DEFAULT_AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS = TimeUnit.SECONDS.toMillis(10); public static final int DEFAULT_AUTO_BALANCER_METRICS_REPORTER_LINGER_MS = (int) TimeUnit.SECONDS.toMillis(1); public static final int DEFAULT_AUTO_BALANCER_METRICS_BATCH_SIZE = 800 * 1000; - public static final boolean DEFAULT_AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE = false; public static final int DEFAULT_AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES = 2; /* Documents */ - public static final String AUTO_BALANCER_BROKER_NW_IN_CAPACITY_DOC = "Maximum network input bandwidth available for the broker in Bytes/s"; - public static final String AUTO_BALANCER_BROKER_NW_OUT_CAPACITY_DOC = "Maximum network output bandwidth available for the broker in Bytes/s"; private static final String AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS_DOC = "Timeout on the Auto Balancer metrics topic creation"; private static final String AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES_DOC = "Number of retries of the Auto Balancer metrics reporter" + " for the topic creation"; @@ -71,31 +63,14 @@ public class AutoBalancerMetricsReporterConfig extends AutoBalancerConfig { + " metrics reporter. Set this config and autobalancer.metrics.reporter.batch.size to a large number to have better batching."; private static final String AUTO_BALANCER_METRICS_REPORTER_BATCH_SIZE_DOC = "The batch.size configuration of KafkaProducer used in AutoBalancer " + " metrics reporter. Set this config and autobalancer.metrics.reporter.linger.ms to a large number to have better batching."; - public static final String AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE_DOC = "Auto balancer metrics reporter will report " - + "metrics using methods that are aware of container boundaries."; static { ProducerConfig.configNames().forEach(name -> CONFIGS.add(PREFIX + name)); - CONFIG.define(AUTO_BALANCER_BROKER_NW_IN_CAPACITY, - ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_BROKER_NW_IN_CAPACITY, - ConfigDef.Importance.HIGH, - AUTO_BALANCER_BROKER_NW_IN_CAPACITY_DOC) - .define(AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, - ConfigDef.Type.DOUBLE, - DEFAULT_AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, - ConfigDef.Importance.HIGH, - AUTO_BALANCER_BROKER_NW_OUT_CAPACITY_DOC) - .define(AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_CONFIG, + CONFIG.define(AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_CONFIG, ConfigDef.Type.LONG, DEFAULT_AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS, ConfigDef.Importance.HIGH, AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_DOC) - .define(AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE_CONFIG, - ConfigDef.Type.BOOLEAN, - DEFAULT_AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE, - ConfigDef.Importance.LOW, - AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE_DOC) .define(AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS_CONFIG, ConfigDef.Type.LONG, DEFAULT_AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS, diff --git a/core/src/main/java/kafka/autobalancer/detector/AnomalyDetector.java b/core/src/main/java/kafka/autobalancer/detector/AnomalyDetector.java index 616172f180..aa7f7aec0e 100644 --- a/core/src/main/java/kafka/autobalancer/detector/AnomalyDetector.java +++ b/core/src/main/java/kafka/autobalancer/detector/AnomalyDetector.java @@ -48,20 +48,16 @@ public class AnomalyDetector { private final long detectInterval; private final long maxTolerateMetricsDelayMs; private final long coolDownIntervalPerActionMs; - private final boolean aggregateBrokerLoad; private volatile boolean running; AnomalyDetector(LogContext logContext, int maxActionsNumPerDetect, long detectIntervalMs, long maxTolerateMetricsDelayMs, - long coolDownIntervalPerActionMs, boolean aggregateBrokerLoad, ClusterModel clusterModel, - ActionExecutorService actionExecutor, List goals, - Set excludedBrokers, Set excludedTopics) { + long coolDownIntervalPerActionMs, ClusterModel clusterModel, ActionExecutorService actionExecutor, + List goals, Set excludedBrokers, Set excludedTopics) { this.logger = logContext.logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); - this.maxActionsNumPerExecution = maxActionsNumPerDetect; this.detectInterval = detectIntervalMs; this.maxTolerateMetricsDelayMs = maxTolerateMetricsDelayMs; this.coolDownIntervalPerActionMs = coolDownIntervalPerActionMs; - this.aggregateBrokerLoad = aggregateBrokerLoad; this.clusterModel = clusterModel; this.actionExecutor = actionExecutor; this.executorService = Executors.newSingleThreadScheduledExecutor(new AutoBalancerThreadFactory("anomaly-detector")); @@ -102,19 +98,20 @@ public void resume() { this.running = true; } - void detect() { + public void detect() { if (!this.running) { return; } logger.info("Start detect"); // The delay in processing kraft log could result in outdated cluster snapshot - ClusterModelSnapshot snapshot = this.clusterModel.snapshot(excludedBrokers, excludedTopics, - this.maxTolerateMetricsDelayMs, this.aggregateBrokerLoad); + ClusterModelSnapshot snapshot = this.clusterModel.snapshot(excludedBrokers, excludedTopics, this.maxTolerateMetricsDelayMs); for (BrokerUpdater.Broker broker : snapshot.brokers()) { - logger.info("Broker status: {}", broker); - for (TopicPartitionReplicaUpdater.TopicPartitionReplica replica : snapshot.replicasFor(broker.getBrokerId())) { - logger.debug("Replica status {}", replica); + logger.info("Broker status: {}", broker.shortString()); + if (logger.isDebugEnabled()) { + for (TopicPartitionReplicaUpdater.TopicPartitionReplica replica : snapshot.replicasFor(broker.getBrokerId())) { + logger.debug("Replica status {}", replica.shortString()); + } } } diff --git a/core/src/main/java/kafka/autobalancer/detector/AnomalyDetectorBuilder.java b/core/src/main/java/kafka/autobalancer/detector/AnomalyDetectorBuilder.java index c4f5b399bf..9296c6256e 100644 --- a/core/src/main/java/kafka/autobalancer/detector/AnomalyDetectorBuilder.java +++ b/core/src/main/java/kafka/autobalancer/detector/AnomalyDetectorBuilder.java @@ -39,7 +39,6 @@ public class AnomalyDetectorBuilder { private long detectIntervalMs = 60000; private long maxTolerateMetricsDelayMs = 30000; private long coolDownIntervalPerActionMs = 100; - private boolean aggregateBrokerLoad = true; public AnomalyDetectorBuilder() { @@ -105,11 +104,6 @@ public AnomalyDetectorBuilder maxTolerateMetricsDelayMs(long maxTolerateMetricsD return this; } - public AnomalyDetectorBuilder aggregateBrokerLoad(boolean aggregateBrokerLoad) { - this.aggregateBrokerLoad = aggregateBrokerLoad; - return this; - } - public AnomalyDetectorBuilder coolDownIntervalPerActionMs(long coolDownIntervalPerActionMs) { this.coolDownIntervalPerActionMs = coolDownIntervalPerActionMs; return this; @@ -129,6 +123,6 @@ public AnomalyDetector build() { throw new IllegalArgumentException("At least one goal must be set"); } return new AnomalyDetector(logContext, maxActionsNumPerDetect, detectIntervalMs, maxTolerateMetricsDelayMs, - coolDownIntervalPerActionMs, aggregateBrokerLoad, clusterModel, executor, goalsByPriority, excludedBrokers, excludedTopics); + coolDownIntervalPerActionMs, clusterModel, executor, goalsByPriority, excludedBrokers, excludedTopics); } } diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java index f55b031264..beb2c5b2cf 100644 --- a/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java @@ -18,10 +18,10 @@ package kafka.autobalancer.goals; import kafka.autobalancer.common.Action; +import kafka.autobalancer.model.BrokerUpdater; import kafka.autobalancer.model.ClusterModelSnapshot; import kafka.autobalancer.model.ModelUtils; -import kafka.autobalancer.model.BrokerUpdater.Broker; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater; import java.util.Objects; import java.util.Set; @@ -41,10 +41,10 @@ public abstract class AbstractGoal implements Goal { * == 0.5 means action with no affection * > 0.5 means positive action */ - private double scoreDelta(Broker srcBrokerBefore, Broker destBrokerBefore, Broker srcBrokerAfter, Broker destBrokerAfter) { + private double scoreDelta(BrokerUpdater.Broker srcBrokerBefore, BrokerUpdater.Broker destBrokerBefore, BrokerUpdater.Broker srcBrokerAfter, BrokerUpdater.Broker destBrokerAfter) { double scoreBefore = Math.min(brokerScore(srcBrokerBefore), brokerScore(destBrokerBefore)); double scoreAfter = Math.min(brokerScore(srcBrokerAfter), brokerScore(destBrokerAfter)); - return GoalUtils.normalize(scoreAfter - scoreBefore, 1.0, -1.0); + return GoalUtils.linearNormalization(scoreAfter - scoreBefore, 1.0, -1.0); } /** @@ -57,10 +57,10 @@ private double scoreDelta(Broker srcBrokerBefore, Broker destBrokerBefore, Broke * @return normalized score. 0 means not allowed action * > 0 means permitted action, but can be positive or negative for this goal */ - double calculateAcceptanceScore(Broker srcBrokerBefore, Broker destBrokerBefore, Broker srcBrokerAfter, Broker destBrokerAfter) { + protected double calculateAcceptanceScore(BrokerUpdater.Broker srcBrokerBefore, BrokerUpdater.Broker destBrokerBefore, BrokerUpdater.Broker srcBrokerAfter, BrokerUpdater.Broker destBrokerAfter) { double score = scoreDelta(srcBrokerBefore, destBrokerBefore, srcBrokerAfter, destBrokerAfter); - if (!isHardGoal()) { + if (type() != GoalType.HARD) { return score; } @@ -88,11 +88,11 @@ public double actionAcceptanceScore(Action action, ClusterModelSnapshot cluster) if (!GoalUtils.isValidAction(action, cluster)) { return 0.0; } - Broker srcBrokerBefore = cluster.broker(action.getSrcBrokerId()); - Broker destBrokerBefore = cluster.broker(action.getDestBrokerId()); - Broker srcBrokerAfter = new Broker(srcBrokerBefore); - Broker destBrokerAfter = new Broker(destBrokerBefore); - TopicPartitionReplica srcReplica = cluster.replica(action.getSrcBrokerId(), action.getSrcTopicPartition()); + BrokerUpdater.Broker srcBrokerBefore = cluster.broker(action.getSrcBrokerId()); + BrokerUpdater.Broker destBrokerBefore = cluster.broker(action.getDestBrokerId()); + BrokerUpdater.Broker srcBrokerAfter = srcBrokerBefore.copy(); + BrokerUpdater.Broker destBrokerAfter = destBrokerBefore.copy(); + TopicPartitionReplicaUpdater.TopicPartitionReplica srcReplica = cluster.replica(action.getSrcBrokerId(), action.getSrcTopicPartition()); switch (action.getType()) { case MOVE: @@ -111,8 +111,8 @@ public double actionAcceptanceScore(Action action, ClusterModelSnapshot cluster) } @Override - public Set getEligibleBrokers(ClusterModelSnapshot cluster) { - return cluster.brokers().stream().filter(Broker::isActive).collect(Collectors.toSet()); + public Set getEligibleBrokers(ClusterModelSnapshot cluster) { + return cluster.brokers().stream().filter(BrokerUpdater.Broker::isActive).collect(Collectors.toSet()); } @Override diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java deleted file mode 100644 index 8ef22378f2..0000000000 --- a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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 kafka.autobalancer.goals; - -import kafka.autobalancer.common.Action; -import kafka.autobalancer.common.ActionType; -import kafka.autobalancer.model.ClusterModelSnapshot; -import kafka.autobalancer.model.BrokerUpdater.Broker; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Set; - -public abstract class AbstractResourceCapacityGoal extends AbstractResourceGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(AbstractResourceCapacityGoal.class); - protected double utilizationThreshold; - - @Override - public boolean isHardGoal() { - return true; - } - - @Override - public List optimize(ClusterModelSnapshot cluster, Collection goalsByPriority) { - List actions = new ArrayList<>(); - validateConfig(); - Set eligibleBrokers = getEligibleBrokers(cluster); - List brokersToOptimize = new ArrayList<>(); - for (Broker broker : eligibleBrokers) { - if (!isBrokerAcceptable(broker)) { - LOGGER.warn("Broker {} violates goal {}", broker.getBrokerId(), name()); - brokersToOptimize.add(broker); - } - } - brokersToOptimize.forEach(eligibleBrokers::remove); - List candidateBrokers = new ArrayList<>(eligibleBrokers); - for (Broker broker : brokersToOptimize) { - if (isBrokerAcceptable(broker)) { - continue; - } - List brokerActions = tryReduceLoadByAction(ActionType.MOVE, cluster, broker, candidateBrokers, goalsByPriority); - if (!isBrokerAcceptable(broker)) { - brokerActions.addAll(tryReduceLoadByAction(ActionType.SWAP, cluster, broker, candidateBrokers, goalsByPriority)); - } - actions.addAll(brokerActions); - if (!isBrokerAcceptable(broker)) { - // broker still violates goal after iterating all partitions - onBalanceFailed(broker); - } - } - return actions; - } - - @Override - public boolean isBrokerAcceptable(Broker broker) { - return broker.utilizationFor(resource()) <= this.utilizationThreshold; - } - - @Override - public void validateConfig() { - this.utilizationThreshold = Math.min(1.0, Math.max(0.0, this.utilizationThreshold)); - } - - @Override - public double brokerScore(Broker broker) { - // use spare utilization as score - double spare = this.utilizationThreshold - (broker.utilizationFor(resource())); - // normalize - return GoalUtils.normalize(spare, this.utilizationThreshold, this.utilizationThreshold - 1); - } -} diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceDistributionGoal.java index 4488df0efc..5f12e88145 100644 --- a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceDistributionGoal.java +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceDistributionGoal.java @@ -17,13 +17,13 @@ package kafka.autobalancer.goals; +import com.automq.stream.utils.LogContext; import kafka.autobalancer.common.Action; import kafka.autobalancer.common.ActionType; -import kafka.autobalancer.common.Resource; +import kafka.autobalancer.common.AutoBalancerConstants; import kafka.autobalancer.model.BrokerUpdater; import kafka.autobalancer.model.ClusterModelSnapshot; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; @@ -32,54 +32,33 @@ import java.util.stream.Collectors; public abstract class AbstractResourceDistributionGoal extends AbstractResourceGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(AbstractResourceDistributionGoal.class); - - protected double utilizationDetectThreshold; - - protected double utilizationAvgDeviation; - private double meanResourceUtil; - private double resourceUtilDistLowerBound; - private double resourceUtilDistUpperBound; - - @Override - public boolean isHardGoal() { - return false; - } - - @Override - public void validateConfig() { - this.utilizationDetectThreshold = Math.min(1.0, Math.max(0.0, this.utilizationDetectThreshold)); - this.utilizationAvgDeviation = Math.abs(utilizationAvgDeviation); - } + private static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); @Override public List optimize(ClusterModelSnapshot cluster, Collection goalsByPriority) { List actions = new ArrayList<>(); - validateConfig(); Set eligibleBrokers = getEligibleBrokers(cluster); - calcUtilizationBound(eligibleBrokers); + calculateResourceBound(eligibleBrokers); List brokersToOptimize = new ArrayList<>(); for (BrokerUpdater.Broker broker : eligibleBrokers) { if (!isBrokerAcceptable(broker)) { - LOGGER.warn("Broker {} violates goal {}", broker.getBrokerId(), name()); + LOGGER.warn("BrokerUpdater.Broker {} violates goal {}", broker.getBrokerId(), name()); brokersToOptimize.add(broker); } } - Resource resource = resource(); for (BrokerUpdater.Broker broker : brokersToOptimize) { if (isBrokerAcceptable(broker)) { continue; } List candidateBrokers = eligibleBrokers.stream().filter(b -> b.getBrokerId() != broker.getBrokerId()).collect(Collectors.toList()); - double loadUtil = broker.utilizationFor(resource); - if (requireLessLoad(loadUtil)) { + if (requireLessLoad(broker)) { List brokerActions = tryReduceLoadByAction(ActionType.MOVE, cluster, broker, candidateBrokers, goalsByPriority); if (!isBrokerAcceptable(broker)) { brokerActions.addAll(tryReduceLoadByAction(ActionType.SWAP, cluster, broker, candidateBrokers, goalsByPriority)); } actions.addAll(brokerActions); - } else if (requireMoreLoad(loadUtil)) { + } else if (requireMoreLoad(broker)) { List brokerActions = tryIncreaseLoadByAction(ActionType.MOVE, cluster, broker, candidateBrokers, goalsByPriority); if (isBrokerAcceptable(broker)) { brokerActions.addAll(tryIncreaseLoadByAction(ActionType.SWAP, cluster, broker, candidateBrokers, goalsByPriority)); @@ -95,33 +74,9 @@ public List optimize(ClusterModelSnapshot cluster, Collection goal return actions; } - private void calcUtilizationBound(Set brokers) { - Resource resource = resource(); - meanResourceUtil = brokers.stream().mapToDouble(e -> e.utilizationFor(resource)).sum() / brokers.size(); - resourceUtilDistLowerBound = Math.max(0, meanResourceUtil * (1 - this.utilizationAvgDeviation)); - resourceUtilDistUpperBound = meanResourceUtil * (1 + this.utilizationAvgDeviation); - } - - private boolean requireLessLoad(double util) { - return util > resourceUtilDistUpperBound; - } + protected abstract void calculateResourceBound(Set brokers); - private boolean requireMoreLoad(double util) { - return util < resourceUtilDistLowerBound; - } + protected abstract boolean requireLessLoad(BrokerUpdater.Broker broker); - @Override - public boolean isBrokerAcceptable(BrokerUpdater.Broker broker) { - double util = broker.utilizationFor(resource()); - if (util < this.utilizationDetectThreshold) { - return true; - } - return !requireLessLoad(util) && !requireMoreLoad(util); - } - - @Override - public double brokerScore(BrokerUpdater.Broker broker) { - double utilMeanDeviationAbs = Math.abs(meanResourceUtil - broker.utilizationFor(resource())); - return GoalUtils.normalize(utilMeanDeviationAbs, 1, 0, true); - } + protected abstract boolean requireMoreLoad(BrokerUpdater.Broker broker); } diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java index 791869b7e4..62d7cd5828 100644 --- a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java @@ -17,14 +17,15 @@ package kafka.autobalancer.goals; +import com.automq.stream.utils.LogContext; import kafka.autobalancer.common.Action; import kafka.autobalancer.common.ActionType; +import kafka.autobalancer.common.AutoBalancerConstants; import kafka.autobalancer.common.Resource; import kafka.autobalancer.model.BrokerUpdater; import kafka.autobalancer.model.ClusterModelSnapshot; import kafka.autobalancer.model.TopicPartitionReplicaUpdater; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.AbstractMap; import java.util.ArrayList; @@ -37,7 +38,7 @@ import java.util.stream.Collectors; public abstract class AbstractResourceGoal extends AbstractGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(AbstractGoal.class); + private static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); protected abstract Resource resource(); @@ -52,9 +53,9 @@ private Optional getAcceptableAction(List> can } private double normalizeGoalsScore(Map scoreMap) { - int totalWeight = scoreMap.keySet().stream().mapToInt(Goal::priority).sum(); + int totalWeight = scoreMap.keySet().stream().mapToInt(e -> e.type().priority()).sum(); return scoreMap.entrySet().stream() - .mapToDouble(entry -> entry.getValue() * (double) entry.getKey().priority() / totalWeight) + .mapToDouble(entry -> entry.getValue() * (double) entry.getKey().type().priority() / totalWeight) .sum(); } @@ -75,7 +76,7 @@ private Optional trySwapPartitionOut(ClusterModelSnapshot cluster, Map scoreMap = new HashMap<>(); for (Goal goal : goalsByPriority) { double score = goal.actionAcceptanceScore(action, cluster); - if (goal.isHardGoal() && score == 0) { + if (goal.type() == GoalType.HARD && score == 0) { isHardGoalViolated = true; break; } @@ -102,7 +103,7 @@ private Optional tryMovePartitionOut(ClusterModelSnapshot cluster, Map scoreMap = new HashMap<>(); for (Goal goal : goalsByPriority) { double score = goal.actionAcceptanceScore(action, cluster); - if (goal.isHardGoal() && score == 0) { + if (goal.type() == GoalType.HARD && score == 0) { isHardGoalViolated = true; break; } @@ -139,7 +140,7 @@ protected List tryReduceLoadByAction(ActionType actionType, .sorted(Comparator.comparingDouble(r -> -r.load(resource()))) // higher load first .collect(Collectors.toList()); for (TopicPartitionReplicaUpdater.TopicPartitionReplica tp : srcReplicas) { - candidateBrokers.sort(Comparator.comparingDouble(b -> b.utilizationFor(resource()))); // lower load first + candidateBrokers.sort(lowLoadComparator()); // lower load first Optional optionalAction; if (actionType == ActionType.MOVE) { optionalAction = tryMovePartitionOut(cluster, tp, srcBroker, candidateBrokers, goalsByPriority); @@ -176,7 +177,7 @@ protected List tryIncreaseLoadByAction(ActionType actionType, List candidateBrokers, Collection goalsByPriority) { List actionList = new ArrayList<>(); - candidateBrokers.sort(Comparator.comparingDouble(b -> -b.utilizationFor(resource()))); // higher load first + candidateBrokers.sort(highLoadComparator()); // higher load first for (BrokerUpdater.Broker candidateBroker : candidateBrokers) { List candidateReplicas = cluster .replicasFor(candidateBroker.getBrokerId()) @@ -204,4 +205,8 @@ protected List tryIncreaseLoadByAction(ActionType actionType, } return actionList; } + + protected abstract Comparator highLoadComparator(); + + protected abstract Comparator lowLoadComparator(); } diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoal.java new file mode 100644 index 0000000000..d4534860aa --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoal.java @@ -0,0 +1,80 @@ +/* + * 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 kafka.autobalancer.goals; + +import kafka.autobalancer.common.Resource; +import kafka.autobalancer.model.BrokerUpdater; + +import java.util.Comparator; +import java.util.Set; + +public abstract class AbstractResourceUsageDistributionGoal extends AbstractResourceDistributionGoal { + private static final double DEFAULT_MAX_LOAD_BYTES = 100 * 1024 * 1024; + private final Comparator highLoadComparator = Comparator.comparingDouble(b -> -b.load(resource())); + private final Comparator lowLoadComparator = Comparator.comparingDouble(b -> b.load(resource())); + + protected long usageDetectThreshold; + + protected double usageAvgDeviation; + private double usageAvg; + private double usageDistLowerBound; + private double usageDistUpperBound; + + @Override + protected void calculateResourceBound(Set brokers) { + Resource resource = resource(); + usageAvg = brokers.stream().mapToDouble(e -> e.load(resource)).sum() / brokers.size(); + usageDistLowerBound = Math.max(0, usageAvg * (1 - this.usageAvgDeviation)); + usageDistUpperBound = usageAvg * (1 + this.usageAvgDeviation); + } + + @Override + protected boolean requireLessLoad(BrokerUpdater.Broker broker) { + return broker.load(resource()) > usageDistUpperBound; + } + + @Override + protected boolean requireMoreLoad(BrokerUpdater.Broker broker) { + return broker.load(resource()) < usageDistLowerBound; + } + + @Override + public boolean isBrokerAcceptable(BrokerUpdater.Broker broker) { + double load = broker.load(resource()); + if (load < this.usageDetectThreshold) { + return true; + } + return load >= usageDistLowerBound && load <= usageDistUpperBound; + } + + @Override + public double brokerScore(BrokerUpdater.Broker broker) { + double loadAvgDeviationAbs = Math.abs(usageAvg - broker.load(resource())); + return GoalUtils.linearNormalization(loadAvgDeviationAbs, DEFAULT_MAX_LOAD_BYTES, 0, true); + } + + @Override + protected Comparator highLoadComparator() { + return highLoadComparator; + } + + @Override + protected Comparator lowLoadComparator() { + return lowLoadComparator; + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/Goal.java b/core/src/main/java/kafka/autobalancer/goals/Goal.java index 34a38b4ef6..9e19a7ca1b 100644 --- a/core/src/main/java/kafka/autobalancer/goals/Goal.java +++ b/core/src/main/java/kafka/autobalancer/goals/Goal.java @@ -28,17 +28,13 @@ public interface Goal extends Configurable, Comparable { - boolean isHardGoal(); - List optimize(ClusterModelSnapshot cluster, Collection goalsByPriority); - void validateConfig(); - void onBalanceFailed(BrokerUpdater.Broker broker); boolean isBrokerAcceptable(BrokerUpdater.Broker broker); - int priority(); + GoalType type(); Set getEligibleBrokers(ClusterModelSnapshot cluster); @@ -57,6 +53,6 @@ public interface Goal extends Configurable, Comparable { @Override default int compareTo(Goal other) { - return Integer.compare(other.priority(), this.priority()); + return Integer.compare(other.type().priority(), this.type().priority()); } } diff --git a/core/src/main/java/kafka/autobalancer/goals/GoalType.java b/core/src/main/java/kafka/autobalancer/goals/GoalType.java new file mode 100644 index 0000000000..685032d897 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/GoalType.java @@ -0,0 +1,33 @@ +/* + * 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 kafka.autobalancer.goals; + +public enum GoalType { + HARD(10), + SOFT(8); + + private final int priority; + + GoalType(int priority) { + this.priority = priority; + } + + public int priority() { + return priority; + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java b/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java index 684ce108c0..fef4cab532 100644 --- a/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java +++ b/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java @@ -36,13 +36,13 @@ public static boolean isValidAction(Action action, ClusterModelSnapshot cluster) return true; } - public static double normalize(double value, double max, double min) { - return normalize(value, max, min, false); + public static double linearNormalization(double value, double max, double min) { + return linearNormalization(value, max, min, false); } - public static double normalize(double value, double max, double min, boolean reverse) { + public static double linearNormalization(double value, double max, double min, boolean reverse) { if (reverse) { - return 1 - (value - min) / (max - min); + return 1 - ((value - min) / (max - min)); } return (value - min) / (max - min); } diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkInCapacityGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkInCapacityGoal.java deleted file mode 100644 index 013ee76114..0000000000 --- a/core/src/main/java/kafka/autobalancer/goals/NetworkInCapacityGoal.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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 kafka.autobalancer.goals; - -import kafka.autobalancer.common.Resource; -import kafka.autobalancer.model.BrokerUpdater; -import kafka.autobalancer.config.AutoBalancerControllerConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -public class NetworkInCapacityGoal extends AbstractResourceCapacityGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(NetworkInCapacityGoal.class); - - @Override - public String name() { - return NetworkInCapacityGoal.class.getSimpleName(); - } - - @Override - protected Resource resource() { - return Resource.NW_IN; - } - - @Override - public void configure(Map configs) { - AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(configs, false); - this.utilizationThreshold = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_UTILIZATION_THRESHOLD); - } - - @Override - public void onBalanceFailed(BrokerUpdater.Broker broker) { - LOGGER.warn("Failed to reduce broker {} network inbound load after iterating all partitions", broker.getBrokerId()); - } - - @Override - public int priority() { - return GoalConstants.NETWORK_CAPACITY_GOAL_PRIORITY; - } -} diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkInUsageDistributionGoal.java similarity index 66% rename from core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java rename to core/src/main/java/kafka/autobalancer/goals/NetworkInUsageDistributionGoal.java index a581a720a2..829107c1fe 100644 --- a/core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkInUsageDistributionGoal.java @@ -17,20 +17,21 @@ package kafka.autobalancer.goals; +import com.automq.stream.utils.LogContext; +import kafka.autobalancer.common.AutoBalancerConstants; import kafka.autobalancer.common.Resource; -import kafka.autobalancer.model.BrokerUpdater; import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.model.BrokerUpdater; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Map; -public class NetworkInDistributionGoal extends AbstractResourceDistributionGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(NetworkInDistributionGoal.class); +public class NetworkInUsageDistributionGoal extends AbstractResourceUsageDistributionGoal { + private static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); @Override public String name() { - return NetworkInDistributionGoal.class.getSimpleName(); + return NetworkInUsageDistributionGoal.class.getSimpleName(); } @Override @@ -41,8 +42,9 @@ protected Resource resource() { @Override public void configure(Map configs) { AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(configs, false); - this.utilizationDetectThreshold = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD); - this.utilizationAvgDeviation = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION); + this.usageDetectThreshold = controllerConfig.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD); + this.usageAvgDeviation = Math.max(0.0, Math.min(1.0, + controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION))); } @Override @@ -51,7 +53,7 @@ public void onBalanceFailed(BrokerUpdater.Broker broker) { } @Override - public int priority() { - return GoalConstants.NETWORK_DISTRIBUTION_GOAL_PRIORITY; + public GoalType type() { + return GoalType.SOFT; } } diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java deleted file mode 100644 index 618db809ab..0000000000 --- a/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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 kafka.autobalancer.goals; - -import kafka.autobalancer.common.Resource; -import kafka.autobalancer.model.BrokerUpdater; -import kafka.autobalancer.config.AutoBalancerControllerConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -public class NetworkOutCapacityGoal extends AbstractResourceCapacityGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(NetworkOutCapacityGoal.class); - - @Override - public String name() { - return NetworkOutCapacityGoal.class.getSimpleName(); - } - - @Override - protected Resource resource() { - return Resource.NW_OUT; - } - - @Override - public void configure(Map configs) { - AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(configs, false); - this.utilizationThreshold = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_UTILIZATION_THRESHOLD); - } - - @Override - public void onBalanceFailed(BrokerUpdater.Broker broker) { - LOGGER.warn("Failed to reduce broker {} network outbound load after iterating all partitions", broker.getBrokerId()); - } - - @Override - public int priority() { - return GoalConstants.NETWORK_CAPACITY_GOAL_PRIORITY; - } -} diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkOutUsageDistributionGoal.java similarity index 66% rename from core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java rename to core/src/main/java/kafka/autobalancer/goals/NetworkOutUsageDistributionGoal.java index 7cf52890f5..adc3504d93 100644 --- a/core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkOutUsageDistributionGoal.java @@ -17,20 +17,21 @@ package kafka.autobalancer.goals; +import com.automq.stream.utils.LogContext; +import kafka.autobalancer.common.AutoBalancerConstants; import kafka.autobalancer.common.Resource; -import kafka.autobalancer.model.BrokerUpdater; import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.model.BrokerUpdater; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Map; -public class NetworkOutDistributionGoal extends AbstractResourceDistributionGoal { - private static final Logger LOGGER = LoggerFactory.getLogger(NetworkOutDistributionGoal.class); +public class NetworkOutUsageDistributionGoal extends AbstractResourceUsageDistributionGoal { + private static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); @Override public String name() { - return NetworkOutDistributionGoal.class.getSimpleName(); + return NetworkOutUsageDistributionGoal.class.getSimpleName(); } @Override @@ -41,8 +42,9 @@ protected Resource resource() { @Override public void configure(Map configs) { AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(configs, false); - this.utilizationDetectThreshold = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD); - this.utilizationAvgDeviation = controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION); + this.usageDetectThreshold = controllerConfig.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD); + this.usageAvgDeviation = Math.max(0.0, Math.min(1.0, + controllerConfig.getDouble(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION))); } @Override @@ -51,7 +53,7 @@ public void onBalanceFailed(BrokerUpdater.Broker broker) { } @Override - public int priority() { - return GoalConstants.NETWORK_DISTRIBUTION_GOAL_PRIORITY; + public GoalType type() { + return GoalType.SOFT; } } diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java index ade23d5176..27ccf9b3ed 100644 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java @@ -27,7 +27,7 @@ import com.yammer.metrics.core.MetricsRegistryListener; import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics; -import kafka.autobalancer.metricsreporter.metric.BrokerMetrics; +import kafka.autobalancer.metricsreporter.metric.MetricClassId; import kafka.autobalancer.metricsreporter.metric.MetricSerde; import kafka.autobalancer.metricsreporter.metric.MetricsUtils; import kafka.autobalancer.metricsreporter.metric.YammerMetricProcessor; @@ -76,9 +76,6 @@ public class AutoBalancerMetricsReporter implements MetricsRegistryListener, Met private int numMetricSendFailure = 0; private volatile boolean shutdown = false; private int metricsReporterCreateRetries; - private boolean kubernetesMode; - private double brokerNwInCapacity; - private double brokerNwOutCapacity; static String getBootstrapServers(Map configs) { Object port = configs.get("port"); @@ -103,16 +100,9 @@ public void init(List metrics) { yammerMetricProcessor = new YammerMetricProcessor(); metricsReporterRunner.start(); metricsRegistry.addListener(this); - addMandatoryBrokerMetrics(); LOGGER.info("AutoBalancerMetricsReporter init successful"); } - private void addMandatoryBrokerMetrics() { - for (String name : MetricsUtils.getMetricNameMaybeMissing()) { - interestedMetrics.putIfAbsent(MetricsUtils.buildBrokerMetricName(name), MetricsUtils.getEmptyMetricFor(name)); - } - } - /** * On new yammer metric added * @@ -189,16 +179,6 @@ public void configure(Map rawConfigs) { AutoBalancerMetricsReporterConfig.config(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } - //Add AUTO_BALANCER_BROKER_NW_IN/OUT_CAPACITY by S3NetworkBaselineBandwidthProp config value if not set - Object s3NetworkBaselineBandwidth = configs.get(KafkaConfig.S3NetworkBaselineBandwidthProp()); - if (s3NetworkBaselineBandwidth != null) { - if (!configs.containsKey(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_IN_CAPACITY)) - configs.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_IN_CAPACITY, s3NetworkBaselineBandwidth); - - if (!configs.containsKey(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_OUT_CAPACITY)) - configs.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, s3NetworkBaselineBandwidth); - } - AutoBalancerMetricsReporterConfig reporterConfig = new AutoBalancerMetricsReporterConfig(configs, false); setIfAbsent(producerProps, @@ -227,12 +207,9 @@ public void configure(Map rawConfigs) { if (brokerRack == null) { brokerRack = ""; } - brokerNwInCapacity = reporterConfig.getDouble(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_IN_CAPACITY); - brokerNwOutCapacity = reporterConfig.getDouble(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_OUT_CAPACITY); autoBalancerMetricsTopic = reporterConfig.getString(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_TOPIC_CONFIG); reportingIntervalMs = reporterConfig.getLong(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_CONFIG); - kubernetesMode = reporterConfig.getBoolean(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_KUBERNETES_MODE_CONFIG); LOGGER.info("AutoBalancerMetricsReporter configuration finished"); } @@ -332,7 +309,6 @@ private void reportMetrics(long now) throws Exception { YammerMetricProcessor.Context context = new YammerMetricProcessor.Context(now, brokerId, brokerRack, reportingIntervalMs); processYammerMetrics(context); - processCpuMetrics(context); for (Map.Entry entry : context.getMetricMap().entrySet()) { sendAutoBalancerMetric(entry.getValue()); } @@ -345,43 +321,20 @@ private void processYammerMetrics(YammerMetricProcessor.Context context) throws LOGGER.trace("Processing yammer metric {}, scope = {}", entry.getKey(), entry.getKey().getScope()); entry.getValue().processWith(yammerMetricProcessor, entry.getKey(), context); } - // add broker capacity info - context.merge(new BrokerMetrics(context.time(), brokerId, brokerRack) - .put(RawMetricType.BROKER_CAPACITY_NW_IN, brokerNwInCapacity) - .put(RawMetricType.BROKER_CAPACITY_NW_OUT, brokerNwOutCapacity)); addMandatoryPartitionMetrics(context); } private void addMandatoryPartitionMetrics(YammerMetricProcessor.Context context) { for (AutoBalancerMetrics metrics : context.getMetricMap().values()) { - if (metrics.metricClassId() == AutoBalancerMetrics.MetricClassId.PARTITION_METRIC + if (metrics.metricClassId() == MetricClassId.PARTITION_METRIC && !MetricsUtils.sanityCheckTopicPartitionMetricsCompleteness(metrics)) { - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.TOPIC_PARTITION_BYTES_IN, 0.0); - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.TOPIC_PARTITION_BYTES_OUT, 0.0); - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.PARTITION_SIZE, 0.0); - } else if (metrics.metricClassId() == AutoBalancerMetrics.MetricClassId.BROKER_METRIC - && !MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)) { - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.ALL_TOPIC_BYTES_IN, 0.0); - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.ALL_TOPIC_BYTES_OUT, 0.0); - metrics.getMetricTypeValueMap().putIfAbsent(RawMetricType.BROKER_CPU_UTIL, 0.0); + metrics.getMetricValueMap().putIfAbsent(RawMetricType.TOPIC_PARTITION_BYTES_IN, 0.0); + metrics.getMetricValueMap().putIfAbsent(RawMetricType.TOPIC_PARTITION_BYTES_OUT, 0.0); + metrics.getMetricValueMap().putIfAbsent(RawMetricType.PARTITION_SIZE, 0.0); } } } - private void processCpuMetrics(YammerMetricProcessor.Context context) { - BrokerMetrics brokerMetrics = null; - try { - brokerMetrics = MetricsUtils.getCpuMetric(context.time(), brokerId, brokerRack, kubernetesMode); - } catch (Exception e) { - LOGGER.error("Create cpu metrics failed: {}", e.getMessage()); - } - if (brokerMetrics == null) { - brokerMetrics = new BrokerMetrics(context.time(), brokerId, brokerRack); - brokerMetrics.put(RawMetricType.BROKER_CPU_UTIL, 0.0); - } - context.merge(brokerMetrics); - } - private void addMetricIfInterested(MetricName name, Metric metric) { LOGGER.debug("Checking Yammer metric {}", name); if (MetricsUtils.isInterested(name)) { diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java index e5d6ff44c8..2535a4a991 100644 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java @@ -34,8 +34,8 @@ * An interface for all the raw metrics reported by {@link AutoBalancerMetricsReporter}. */ public abstract class AutoBalancerMetrics { - static final byte METRIC_VERSION = 0; - private final Map metricTypeValueMap = new HashMap<>(); + protected static final byte METRIC_VERSION = 0; + private final Map metricValueMap = new HashMap<>(); private final long time; private final int brokerId; private final String brokerRack; @@ -44,11 +44,11 @@ public AutoBalancerMetrics(long time, int brokerId, String brokerRack) { this(time, brokerId, brokerRack, Collections.emptyMap()); } - public AutoBalancerMetrics(long time, int brokerId, String brokerRack, Map metricTypeValueMap) { + public AutoBalancerMetrics(long time, int brokerId, String brokerRack, Map metricValueMap) { this.time = time; this.brokerId = brokerId; this.brokerRack = brokerRack; - this.metricTypeValueMap.putAll(metricTypeValueMap); + this.metricValueMap.putAll(metricValueMap); } static Map parseMetricsMap(ByteBuffer buffer) { @@ -63,18 +63,18 @@ static Map parseMetricsMap(ByteBuffer buffer) { } public AutoBalancerMetrics put(RawMetricType type, double value) { - this.metricTypeValueMap.put(type, value); + this.metricValueMap.put(type, value); return this; } public void add(AutoBalancerMetrics metrics) { - for (Map.Entry metricEntry : metrics.metricTypeValueMap.entrySet()) { - this.metricTypeValueMap.putIfAbsent(metricEntry.getKey(), metricEntry.getValue()); + for (Map.Entry metricEntry : metrics.metricValueMap.entrySet()) { + this.metricValueMap.putIfAbsent(metricEntry.getKey(), metricEntry.getValue()); } } - public Map getMetricTypeValueMap() { - return metricTypeValueMap; + public Map getMetricValueMap() { + return metricValueMap; } public abstract String key(); @@ -104,12 +104,12 @@ public String brokerRack() { } public int bodySize() { - return Integer.SIZE + (Byte.SIZE + Double.SIZE) * metricTypeValueMap.size(); + return Integer.SIZE + (Byte.SIZE + Double.SIZE) * metricValueMap.size(); } public ByteBuffer writeBody(ByteBuffer buffer) { - buffer.putInt(metricTypeValueMap.size()); - for (Map.Entry entry : metricTypeValueMap.entrySet()) { + buffer.putInt(metricValueMap.size()); + for (Map.Entry entry : metricValueMap.entrySet()) { buffer.put(entry.getKey().id()); buffer.putDouble(entry.getValue()); } @@ -126,7 +126,7 @@ public ByteBuffer writeBody(ByteBuffer buffer) { public String buildKVString() { StringBuilder builder = new StringBuilder(); - for (Map.Entry entry : metricTypeValueMap.entrySet()) { + for (Map.Entry entry : metricValueMap.entrySet()) { builder.append(entry.getKey()); builder.append(":"); builder.append(String.format("%.4f", entry.getValue())); @@ -138,30 +138,4 @@ public String buildKVString() { public String toString() { return String.format("[BrokerId=%d,Time=%d,Key:Value=%s]", brokerId, time, buildKVString()); } - - /** - * An enum that list all the implementations of the interface. This id will be store in the serialized - * metrics to help the metric sampler to decide using which class to deserialize the metric bytes. - */ - public enum MetricClassId { - BROKER_METRIC((byte) 0), PARTITION_METRIC((byte) 1); - - private final byte id; - - MetricClassId(byte id) { - this.id = id; - } - - static MetricClassId forId(byte id) { - if (id < values().length) { - return values()[id]; - } else { - throw new IllegalArgumentException("MetricClassId " + id + " does not exist."); - } - } - - byte id() { - return id; - } - } } diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java deleted file mode 100644 index f89bb7b905..0000000000 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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. - */ -/* - * Copyright 2017 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. - */ - -package kafka.autobalancer.metricsreporter.metric; - -import java.io.BufferedReader; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; - -/** - * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.ContainerMetricUtils. - */ -public final class ContainerMetricUtils { - // A CPU quota value of -1 indicates that the cgroup does not adhere to any CPU time restrictions - public static final int NO_CPU_QUOTA = -1; - // Paths used to get cgroup information - private static final String QUOTA_PATH = "/sys/fs/cgroup/cpu/cpu.cfs_quota_us"; - private static final String PERIOD_PATH = "/sys/fs/cgroup/cpu/cpu.cfs_period_us"; - // Unix command to execute inside a Linux container to get the number of logical processors available to the node - private static final String NPROC = "nproc"; - - private ContainerMetricUtils() { - } - - /** - * Reads cgroups CPU period from cgroups file. Value has a lowerbound of 1 millisecond and an upperbound of 1 second - * according to https://www.kernel.org/doc/Documentation/scheduler/sched-bwc.txt - * - * @return Cgroups CPU period in microseconds as a double. - */ - private static double getCpuPeriod() throws IOException { - return Double.parseDouble(readFile(CgroupFiles.PERIOD_PATH.getValue())); - } - - /** - * Reads cgroups CPU quota from cgroups file. The value has a lowerbound of 1 millisecond - * according to https://www.kernel.org/doc/Documentation/scheduler/sched-bwc.txt - * - * @return Cgroups CPU quota in microseconds as a double. - */ - private static double getCpuQuota() throws IOException { - return Double.parseDouble(readFile(CgroupFiles.QUOTA_PATH.getValue())); - } - - /** - * Gets the the number of logical cores available to the node. - *

- * We can get this value while running in a container by using the "nproc" command. - * Using other methods like OperatingSystemMXBean.getAvailableProcessors() and - * Runtime.getRuntime().availableProcessors() would require disabling container - * support (-XX:-UseContainerSupport) since these methods are aware of container - * boundaries - * - * @return Number of logical processors on node - */ - private static int getAvailableProcessors() throws IOException { - InputStream in = Runtime.getRuntime().exec(NPROC).getInputStream(); - return Integer.parseInt(readInputStream(in)); - } - - private static String readFile(String path) throws IOException { - return readInputStream(new FileInputStream(path)); - } - - private static String readInputStream(InputStream in) throws IOException { - BufferedReader br = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); - String stream = br.readLine(); - if (stream != null) { - return stream; - } else { - throw new IllegalArgumentException("Nothing was read from stream " + in); - } - } - - /** - * Get the "recent CPU usage" for the JVM process running inside of a container. - *

- * At this time, the methods of OperatingSystemMXBean used for retrieving recent CPU usage are not - * container aware and calculate CPU usage with respect to the physical host instead of the operating - * environment from which they are called from. There have been efforts to make these methods container - * aware but the changes have not been backported to Java versions less than version 14. - *

- * Once these changes get backported, https://bugs.openjdk.java.net/browse/JDK-8226575, we can use - * "getSystemCpuLoad()" for retrieving the CPU usage values when running in a container environment. - * - * @param cpuUtil The "recent CPU usage" for a JVM process with respect to node - * @return the "recent CPU usage" for a JVM process with respect to operating environment - * as a double in [0.0,1.0]. - */ - public static double getContainerProcessCpuLoad(double cpuUtil) throws IOException { - int logicalProcessorsOfNode = getAvailableProcessors(); - double cpuQuota = getCpuQuota(); - if (cpuQuota == NO_CPU_QUOTA) { - return cpuUtil; - } - - // Get the number of CPUs of a node that can be used by the operating environment - double cpuLimit = cpuQuota / getCpuPeriod(); - - // Get the minimal number of CPUs needed to achieve the reported CPU utilization - double cpus = cpuUtil * logicalProcessorsOfNode; - - /* Calculate the CPU utilization of a JVM process with respect to the operating environment. - * Since the operating environment will only use the CPU resources allocated by CGroups, - * it will always be that: cpuLimit >= cpus and the result is in the [0.0,1.0] interval. - */ - return cpus / cpuLimit; - } - - private enum CgroupFiles { - QUOTA_PATH(ContainerMetricUtils.QUOTA_PATH), - PERIOD_PATH(ContainerMetricUtils.PERIOD_PATH); - - private final String value; - - CgroupFiles(String value) { - this.value = value; - } - - private String getValue() { - return value; - } - } -} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricClassId.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricClassId.java new file mode 100644 index 0000000000..79072cc120 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricClassId.java @@ -0,0 +1,44 @@ +/* + * 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 kafka.autobalancer.metricsreporter.metric; + +/** + * An enum that list all the implementations of the interface. This id will be store in the serialized + * metrics to help the metric sampler to decide using which class to deserialize the metric bytes. + */ +public enum MetricClassId { + PARTITION_METRIC((byte) 0), BROKER_METRIC((byte) 1); + + private final byte id; + + MetricClassId(byte id) { + this.id = id; + } + + static MetricClassId forId(byte id) { + if (id < values().length) { + return values()[id]; + } else { + throw new IllegalArgumentException("MetricClassId " + id + " does not exist."); + } + } + + public byte id() { + return id; + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java index fc077608ad..47413107cd 100644 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java @@ -56,13 +56,14 @@ public static byte[] toBytes(AutoBalancerMetrics metric) { */ public static AutoBalancerMetrics fromBytes(byte[] bytes) throws UnknownVersionException { ByteBuffer buffer = ByteBuffer.wrap(bytes); - switch (AutoBalancerMetrics.MetricClassId.forId(buffer.get())) { + + switch (MetricClassId.forId(buffer.get())) { case BROKER_METRIC: return BrokerMetrics.fromBuffer(buffer); case PARTITION_METRIC: return TopicPartitionMetrics.fromBuffer(buffer); default: - // This could happen when a new type of metric is added but we are still running the old code. + // This could happen when a new type of metric is added, but we are still running the old code. // simply ignore the metric by returning a null. return null; } diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java index 8c2f4ec96d..2b040d5d21 100644 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java @@ -24,11 +24,8 @@ import com.yammer.metrics.core.Metric; import com.yammer.metrics.core.MetricName; import kafka.metrics.KafkaMetricsGroup$; -import scala.collection.immutable.Map$; import scala.jdk.javaapi.CollectionConverters; -import java.io.IOException; -import java.lang.management.ManagementFactory; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -58,7 +55,6 @@ public final class MetricsUtils { // Type private static final String LOG_GROUP = "Log"; private static final String BROKER_TOPIC_PARTITION_METRICS_GROUP = "BrokerTopicPartitionMetrics"; - private static final String BROKER_TOPIC_METRICS_GROUP = "BrokerTopicMetrics"; // Name Set. private static final Set INTERESTED_TOPIC_PARTITION_METRIC_NAMES = Set.of(BYTES_IN_PER_SEC, BYTES_OUT_PER_SEC); @@ -84,19 +80,6 @@ public static Metric getEmptyMetricFor(String name) { } } - public static MetricName buildBrokerMetricName(String name) { - String group = null; - String type = null; - if (BYTES_IN_PER_SEC.equals(name) || BYTES_OUT_PER_SEC.equals(name)) { - group = KAFKA_SERVER; - type = BROKER_TOPIC_METRICS_GROUP; - } - if (group == null) { - return null; - } - return KafkaMetricsGroup$.MODULE$.explicitMetricName(group, type, name, Map$.MODULE$.empty()); - } - public static MetricName buildTopicPartitionMetricName(String name, String topic, String partition) { Map tags = Map.of("topic", topic, "partition", partition); String group = null; @@ -202,30 +185,6 @@ private static AutoBalancerMetrics toAutoBalancerMetric(long nowMs, } } - /** - * Get the "recent CPU usage" for the JVM process. - * - * @param nowMs The current time in milliseconds. - * @param brokerId Broker Id. - * @param brokerRack Broker rack. - * @param kubernetesMode If {@code true}, gets CPU usage values with respect to the operating environment instead of node. - * @return the "recent CPU usage" for the JVM process as a double in [0.0,1.0]. - */ - public static BrokerMetrics getCpuMetric(long nowMs, int brokerId, String brokerRack, boolean kubernetesMode) throws IOException { - double cpuUtil = ((com.sun.management.OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean()).getProcessCpuLoad(); - - if (kubernetesMode) { - cpuUtil = ContainerMetricUtils.getContainerProcessCpuLoad(cpuUtil); - } - - if (cpuUtil < 0) { - throw new IOException("Java Virtual Machine recent CPU usage is not available."); - } - BrokerMetrics brokerMetric = new BrokerMetrics(nowMs, brokerId, brokerRack); - brokerMetric.put(RawMetricType.BROKER_CPU_UTIL, cpuUtil); - return brokerMetric; - } - /** * Check whether the yammer metric name is an interested metric. * @@ -250,8 +209,6 @@ private static boolean isInterested(String group, String name, String type, Map< if (group.equals(KAFKA_SERVER)) { if (BROKER_TOPIC_PARTITION_METRICS_GROUP.equals(type)) { return INTERESTED_TOPIC_PARTITION_METRIC_NAMES.contains(name) && sanityCheckTopicPartitionTags(tags); - } else if (BROKER_TOPIC_METRICS_GROUP.equals(type)) { - return INTERESTED_TOPIC_PARTITION_METRIC_NAMES.contains(name) && tags.isEmpty(); } } else if (group.startsWith(KAFKA_LOG_PREFIX) && INTERESTED_LOG_METRIC_NAMES.contains(name)) { return LOG_GROUP.equals(type); @@ -297,8 +254,6 @@ private static AutoBalancerMetrics bytesInToMetric(String topic, int partition, if (topic != null && partition != -1) { return new TopicPartitionMetrics(nowMs, brokerId, brokerRack, topic, partition).put(RawMetricType.TOPIC_PARTITION_BYTES_IN, value); - } else if (topic == null && partition == -1) { - return new BrokerMetrics(nowMs, brokerId, brokerRack).put(RawMetricType.ALL_TOPIC_BYTES_IN, value); } return null; } @@ -308,17 +263,11 @@ private static AutoBalancerMetrics bytesOutToMetric(String topic, int partition, if (topic != null && partition != -1) { return new TopicPartitionMetrics(nowMs, brokerId, brokerRack, topic, partition).put(RawMetricType.TOPIC_PARTITION_BYTES_OUT, value); - } else if (topic == null && partition == -1) { - return new BrokerMetrics(nowMs, brokerId, brokerRack).put(RawMetricType.ALL_TOPIC_BYTES_OUT, value); } return null; } - public static boolean sanityCheckBrokerMetricsCompleteness(AutoBalancerMetrics metrics) { - return metrics.getMetricTypeValueMap().keySet().containsAll(RawMetricType.brokerMetricTypes()); - } - public static boolean sanityCheckTopicPartitionMetricsCompleteness(AutoBalancerMetrics metrics) { - return metrics.getMetricTypeValueMap().keySet().containsAll(RawMetricType.partitionMetricTypes()); + return metrics.getMetricValueMap().keySet().containsAll(RawMetricType.partitionMetricTypes()); } } diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java index 257c825946..bfe17859c6 100644 --- a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java @@ -33,7 +33,6 @@ * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.PartitionMetric. */ public class TopicPartitionMetrics extends AutoBalancerMetrics { - private static final byte METRIC_VERSION = 0; private final String topic; private final int partition; @@ -113,6 +112,7 @@ public int partition() { * @param headerPos Header position * @return Byte buffer of the partition metric. */ + @Override public ByteBuffer toBuffer(int headerPos) { byte[] brokerRackBytes = brokerRack().getBytes(StandardCharsets.UTF_8); byte[] topic = topic().getBytes(StandardCharsets.UTF_8); diff --git a/core/src/main/java/kafka/autobalancer/model/AbstractInstanceUpdater.java b/core/src/main/java/kafka/autobalancer/model/AbstractInstanceUpdater.java new file mode 100644 index 0000000000..aa4a125fd7 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/AbstractInstanceUpdater.java @@ -0,0 +1,186 @@ +/* + * 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 kafka.autobalancer.model; + + +import com.automq.stream.utils.LogContext; +import kafka.autobalancer.common.AutoBalancerConstants; +import kafka.autobalancer.common.RawMetricType; +import kafka.autobalancer.common.Resource; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public abstract class AbstractInstanceUpdater { + protected static final Logger LOGGER = new LogContext().logger(AutoBalancerConstants.AUTO_BALANCER_LOGGER_CLAZZ); + protected final Lock lock = new ReentrantLock(); + + protected abstract boolean validateMetrics(Map metricsMap); + + protected abstract AbstractInstance instance(); + + public boolean update(Map metricsMap, long time) { + if (!validateMetrics(metricsMap)) { + LOGGER.error("Metrics validation failed for: {}, metrics: {}", this.instance().name(), metricsMap.keySet()); + return false; + } + + lock.lock(); + try { + if (time < this.instance().getTimestamp()) { + LOGGER.warn("Metrics for {} is outdated at {}, last updated time {}", this.instance().name(), time, + this.instance().getTimestamp()); + return false; + } + this.instance().update(metricsMap, time); + } finally { + lock.unlock(); + } + LOGGER.debug("Successfully updated on {} at time {}", this.instance().name(), this.instance().getTimestamp()); + return true; + } + + public AbstractInstance get() { + return get(-1); + } + + public AbstractInstance get(long timeSince) { + lock.lock(); + try { + if (this.instance().getTimestamp() < timeSince) { + LOGGER.debug("Metrics for {} is out of sync, expected earliest time: {}, actual: {}", + this.instance().name(), timeSince, this.instance().getTimestamp()); + return null; + } + if (!isValidInstance()) { + LOGGER.debug("Metrics for {} is invalid", this.instance().name()); + return null; + } + return this.instance().copy(); + } finally { + lock.unlock(); + } + } + + protected abstract boolean isValidInstance(); + + public static abstract class AbstractInstance { + protected final double[] loads = new double[Resource.cachedValues().size()]; + protected final Set resources = new HashSet<>(); + protected Map metricsMap = new HashMap<>(); + protected long timestamp = 0L; + + public AbstractInstance() { + + } + + public AbstractInstance(AbstractInstance other) { + System.arraycopy(other.loads, 0, this.loads, 0, loads.length); + this.resources.addAll(other.resources); + this.metricsMap.putAll(other.metricsMap); + this.timestamp = other.timestamp; + } + + public abstract AbstractInstance copy(); + + public abstract void processMetrics(); + + public void setLoad(Resource resource, double value) { + this.resources.add(resource); + this.loads[resource.id()] = value; + } + + public double load(Resource resource) { + if (!this.resources.contains(resource)) { + return 0.0; + } + return this.loads[resource.id()]; + } + + public Set getResources() { + return this.resources; + } + + public void update(Map metricsMap, long timestamp) { + this.metricsMap = metricsMap; + this.timestamp = timestamp; + } + + public Map getMetricsMap() { + return this.metricsMap; + } + + public double ofValue(RawMetricType metricType) { + return this.metricsMap.getOrDefault(metricType, 0.0); + } + + public long getTimestamp() { + return this.timestamp; + } + + protected abstract String name(); + + protected String timeString() { + return "timestamp=" + timestamp; + } + + protected String loadString() { + StringBuilder builder = new StringBuilder(); + builder.append("Loads={"); + for (int i = 0; i < loads.length; i++) { + builder.append(Resource.of(i).resourceString(loads[i])); + if (i != loads.length - 1) { + builder.append(", "); + } + } + builder.append("}"); + return builder.toString(); + } + + protected String metricsString() { + StringBuilder builder = new StringBuilder(); + builder.append("Metrics={"); + int i = 0; + for (Map.Entry entry : metricsMap.entrySet()) { + builder.append(entry.getKey()) + .append("=") + .append(entry.getValue()); + if (i != metricsMap.size() - 1) { + builder.append(", "); + } + i++; + } + builder.append("}"); + return builder.toString(); + } + + @Override + public String toString() { + return timeString() + + ", " + + loadString() + + ", " + + metricsString(); + } + } +} diff --git a/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java b/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java index 78a75f6fb9..4f1f1f19e8 100644 --- a/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java +++ b/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java @@ -19,102 +19,65 @@ import kafka.autobalancer.common.RawMetricType; import kafka.autobalancer.common.Resource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Objects; -import java.util.Set; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -public class BrokerUpdater { - private static final Logger LOGGER = LoggerFactory.getLogger(BrokerUpdater.class); - private final Lock lock = new ReentrantLock(); + +public class BrokerUpdater extends AbstractInstanceUpdater { private final Broker broker; - public BrokerUpdater(int brokerId) { - this.broker = new Broker(brokerId); + public BrokerUpdater(int brokerId, boolean active) { + this.broker = createBroker(brokerId, active); + } + + public void setActive(boolean active) { + lock.lock(); + try { + this.broker.setActive(active); + } finally { + lock.unlock(); + } + } + + protected Broker createBroker(int brokerId, boolean active) { + return new Broker(brokerId, active); + } + + @Override + protected boolean validateMetrics(Map metricsMap) { + return true; + } + + @Override + protected AbstractInstance instance() { + return this.broker; } - public static class Broker { + @Override + protected boolean isValidInstance() { + return broker.isActive(); + } + + public static class Broker extends AbstractInstance { private final int brokerId; - private final double[] brokerCapacity = new double[Resource.cachedValues().size()]; - private final double[] brokerLoad = new double[Resource.cachedValues().size()]; - private final Set resources = new HashSet<>(); - private final Map metricsMap = new HashMap<>(); private boolean active; - private long timestamp; - public Broker(int brokerId) { + public Broker(int brokerId, boolean active) { this.brokerId = brokerId; - Arrays.fill(this.brokerCapacity, Resource.IGNORED_CAPACITY_VALUE); + this.active = active; } public Broker(Broker other) { + super(other); this.brokerId = other.brokerId; - System.arraycopy(other.brokerCapacity, 0, this.brokerCapacity, 0, other.brokerCapacity.length); - System.arraycopy(other.brokerLoad, 0, this.brokerLoad, 0, other.brokerLoad.length); - this.resources.addAll(other.resources); this.active = other.active; - this.timestamp = other.timestamp; } public int getBrokerId() { return this.brokerId; } - public void setCapacity(Resource resource, double value) { - this.brokerCapacity[resource.id()] = value; - } - - public double capacity(Resource resource) { - return this.brokerCapacity[resource.id()]; - } - - public Set getResources() { - return resources; - } - - public void setLoad(Resource resource, double value) { - resources.add(resource); - this.brokerLoad[resource.id()] = value; - } - - public void setMetricValue(RawMetricType metricType, double value) { - this.metricsMap.put(metricType, value); - } - - public double load(Resource resource) { - if (!resources.contains(resource)) { - return 0.0; - } - return this.brokerLoad[resource.id()]; - } - - public void reduceLoad(Resource resource, double delta) { - this.brokerLoad[resource.id()] -= delta; - } - - public void addLoad(Resource resource, double delta) { - resources.add(resource); - this.brokerLoad[resource.id()] += delta; - } - - public double utilizationFor(Resource resource) { - double capacity = capacity(resource); - if (capacity == Resource.IGNORED_CAPACITY_VALUE) { - return 0.0; - } - if (capacity == 0.0) { - return Double.MAX_VALUE; - } - return load(resource) / capacity(resource); - } - public void setActive(boolean active) { this.active = active; } @@ -123,12 +86,12 @@ public boolean isActive() { return this.active; } - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; + public void reduceLoad(Resource resource, double delta) { + this.setLoad(resource, load(resource) - delta); } - public long getTimestamp() { - return this.timestamp; + public void addLoad(Resource resource, double delta) { + this.setLoad(resource, load(resource) + delta); } @Override @@ -144,133 +107,37 @@ public int hashCode() { return Objects.hashCode(brokerId); } - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append("{brokerId=") - .append(brokerId) - .append(", active=").append(active) - .append(", timestamp=").append(timestamp) - .append(", Capacities=["); - for (int i = 0; i < brokerCapacity.length; i++) { - builder.append(Resource.of(i).resourceString(brokerCapacity[i])); - if (i != brokerCapacity.length - 1) { - builder.append(", "); - } - } - builder.append("], Loads=["); - for (int i = 0; i < brokerLoad.length; i++) { - builder.append(Resource.of(i).resourceString(brokerLoad[i])); - if (i != brokerLoad.length - 1) { - builder.append(", "); - } - } - builder.append("]"); - int i = 0; - for (Map.Entry entry : metricsMap.entrySet()) { - if (i == 0) { - builder.append(" Metrics={"); - } - builder.append(entry.getKey()) - .append("=") - .append(entry.getValue()); - if (i != metricsMap.size() - 1) { - builder.append(", "); - } - i++; - } - builder.append("}"); - return builder.toString(); - } - } - - public boolean update(Map metricsMap, long time) { - if (!metricsMap.keySet().containsAll(RawMetricType.brokerMetricTypes())) { - LOGGER.error("Broker metrics for broker={} sanity check failed, metrics is incomplete {}", broker.getBrokerId(), metricsMap.keySet()); - return false; + public String shortString() { + return "Broker{" + + "brokerId=" + brokerId + + ", active=" + active + + ", " + timeString() + + ", " + loadString() + + "}"; } - lock.lock(); - try { - if (time < broker.getTimestamp()) { - LOGGER.warn("Outdated broker metrics at time {} for broker={}, last updated time {}", time, broker.getBrokerId(), broker.getTimestamp()); - return false; - } - for (Map.Entry entry : metricsMap.entrySet()) { - if (entry.getKey().metricScope() != RawMetricType.MetricScope.BROKER) { - continue; - } - switch (entry.getKey()) { - case BROKER_CAPACITY_NW_IN: - broker.setCapacity(Resource.NW_IN, entry.getValue()); - break; - case BROKER_CAPACITY_NW_OUT: - broker.setCapacity(Resource.NW_OUT, entry.getValue()); - break; - case ALL_TOPIC_BYTES_IN: - broker.setLoad(Resource.NW_IN, entry.getValue()); - break; - case ALL_TOPIC_BYTES_OUT: - broker.setLoad(Resource.NW_OUT, entry.getValue()); - break; - case BROKER_CPU_UTIL: - broker.setLoad(Resource.CPU, entry.getValue()); - break; - default: - broker.setMetricValue(entry.getKey(), entry.getValue()); - break; - } - } - broker.setTimestamp(time); - } finally { - lock.unlock(); + @Override + public Broker copy() { + return new Broker(this); } - LOGGER.debug("Successfully updated on broker {} at time {}", broker.getBrokerId(), broker.getTimestamp()); - return true; - } - - public Broker get() { - Broker broker; - lock.lock(); - try { - broker = new Broker(this.broker); - } finally { - lock.unlock(); + @Override + public void processMetrics() { + // do nothing } - return broker; - } - public Broker get(long timeSince) { - Broker broker; - lock.lock(); - try { - // Broker is fenced or in shutdown. - // For fenced or shutting-down brokers, the replicationControlManager will handle the movement of partitions. - // So we do not need to consider them in auto-balancer. - if (!this.broker.isActive()) { - return null; - } - if (this.broker.timestamp < timeSince) { - LOGGER.warn("Broker {} metrics is out of sync, expected earliest time: {}, actual: {}", - this.broker.getBrokerId(), timeSince, this.broker.timestamp); - return null; - } - broker = new Broker(this.broker); - } finally { - lock.unlock(); + @Override + protected String name() { + return "broker-" + brokerId; } - return broker; - } - public void setActive(boolean active) { - lock.lock(); - try { - this.broker.setActive(active); - } finally { - lock.unlock(); + @Override + public String toString() { + return "Broker{" + + "brokerId=" + brokerId + + ", active=" + active + + ", " + super.toString() + + "}"; } } - - } diff --git a/core/src/main/java/kafka/autobalancer/model/ClusterModel.java b/core/src/main/java/kafka/autobalancer/model/ClusterModel.java index 1014479cdf..55a0d2e831 100644 --- a/core/src/main/java/kafka/autobalancer/model/ClusterModel.java +++ b/core/src/main/java/kafka/autobalancer/model/ClusterModel.java @@ -37,7 +37,6 @@ public class ClusterModel { protected final Logger logger; private static final String DEFAULT_RACK_ID = "rack_default"; private static final long DEFAULT_MAX_TOLERATED_METRICS_DELAY_MS = 60000L; - private static final boolean DEFAULT_AGGREGATE_BROKER_LOAD = true; /* * Guard the change on cluster structure (add/remove for brokers, replicas) @@ -63,54 +62,60 @@ public ClusterModel(LogContext logContext) { } public ClusterModelSnapshot snapshot() { - return snapshot(Collections.emptySet(), Collections.emptySet(), DEFAULT_MAX_TOLERATED_METRICS_DELAY_MS, DEFAULT_AGGREGATE_BROKER_LOAD); + return snapshot(Collections.emptySet(), Collections.emptySet(), DEFAULT_MAX_TOLERATED_METRICS_DELAY_MS); } - public ClusterModelSnapshot snapshot(Set excludedBrokerIds, Set excludedTopics, - long maxToleratedMetricsDelay, boolean aggregateBrokerLoad) { + public ClusterModelSnapshot snapshot(Set excludedBrokerIds, Set excludedTopics, long maxToleratedMetricsDelay) { ClusterModelSnapshot snapshot = new ClusterModelSnapshot(); clusterLock.lock(); try { long now = System.currentTimeMillis(); - for (BrokerUpdater brokerUpdater : brokerMap.values()) { - BrokerUpdater.Broker broker = brokerUpdater.get(now - maxToleratedMetricsDelay); + for (Map.Entry entry : brokerMap.entrySet()) { + int brokerId = entry.getKey(); + BrokerUpdater.Broker broker = (BrokerUpdater.Broker) entry.getValue().get(); if (broker == null) { continue; } - if (excludedBrokerIds.contains(broker.getBrokerId())) { + if (excludedBrokerIds.contains(brokerId)) { continue; } - snapshot.addBroker(brokerIdToRackMap.get(broker.getBrokerId()), broker); + broker.processMetrics(); + snapshot.addBroker(brokerId, brokerIdToRackMap.get(brokerId), broker); } for (Map.Entry> entry : brokerReplicaMap.entrySet()) { int brokerId = entry.getKey(); if (snapshot.broker(brokerId) == null) { continue; } - for (TopicPartitionReplicaUpdater replicaUpdater : entry.getValue().values()) { - TopicPartitionReplicaUpdater.TopicPartitionReplica replica = replicaUpdater.get(now - maxToleratedMetricsDelay); + for (Map.Entry tpEntry : entry.getValue().entrySet()) { + TopicPartition tp = tpEntry.getKey(); + TopicPartitionReplicaUpdater.TopicPartitionReplica replica = + (TopicPartitionReplicaUpdater.TopicPartitionReplica) tpEntry.getValue().get(now - maxToleratedMetricsDelay); if (replica == null) { - logger.warn("Broker {} has out of sync topic-partition {}, will be ignored in this round", brokerId, replicaUpdater.topicPartition()); + logger.warn("Broker {} has out of sync topic-partition {}, will be ignored in this round", brokerId, tp); snapshot.removeBroker(brokerIdToRackMap.get(brokerId), brokerId); break; } - if (excludedTopics.contains(replica.getTopicPartition().topic())) { + if (excludedTopics.contains(tp.topic())) { continue; } - snapshot.addTopicPartition(brokerId, replica); + replica.processMetrics(); + snapshot.addTopicPartition(brokerId, tp, replica); } } } finally { clusterLock.unlock(); } - if (aggregateBrokerLoad) { - snapshot.aggregate(); - } + postProcess(snapshot); return snapshot; } + public void postProcess(ClusterModelSnapshot snapshot) { + snapshot.aggregate(); + } + public boolean updateBrokerMetrics(int brokerId, Map metricsMap, long time) { BrokerUpdater brokerUpdater = null; clusterLock.lock(); @@ -148,8 +153,7 @@ public void registerBroker(int brokerId, String rackId) { if (brokerMap.containsKey(brokerId)) { return; } - BrokerUpdater brokerUpdater = new BrokerUpdater(brokerId); - brokerUpdater.setActive(true); + BrokerUpdater brokerUpdater = createBrokerUpdater(brokerId); if (Utils.isBlank(rackId)) { rackId = DEFAULT_RACK_ID; } @@ -161,6 +165,10 @@ public void registerBroker(int brokerId, String rackId) { } } + public BrokerUpdater createBrokerUpdater(int brokerId) { + return new BrokerUpdater(brokerId, true); + } + public void unregisterBroker(int brokerId) { clusterLock.lock(); try { @@ -233,12 +241,16 @@ public void createPartition(Uuid topicId, int partitionId, int brokerId) { } topicPartitionReplicaMap.get(topicName).put(partitionId, brokerId); TopicPartition tp = new TopicPartition(topicName, partitionId); - brokerReplicaMap.get(brokerId).put(tp, new TopicPartitionReplicaUpdater(tp)); + brokerReplicaMap.get(brokerId).put(tp, createReplicaUpdater(tp)); } finally { clusterLock.unlock(); } } + public TopicPartitionReplicaUpdater createReplicaUpdater(TopicPartition tp) { + return new TopicPartitionReplicaUpdater(tp); + } + public void reassignPartition(Uuid topicId, int partitionId, int brokerId) { clusterLock.lock(); try { diff --git a/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java b/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java index fa09568f8c..b81a774442 100644 --- a/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java +++ b/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java @@ -20,8 +20,6 @@ import kafka.autobalancer.common.Action; import kafka.autobalancer.common.ActionType; import kafka.autobalancer.common.Resource; -import kafka.autobalancer.model.BrokerUpdater.Broker; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; import org.apache.kafka.common.TopicPartition; import java.util.Collection; @@ -31,8 +29,8 @@ public class ClusterModelSnapshot { private final Map rackToBrokerMap; - private final Map brokerMap; - private final Map> brokerToReplicaMap; + private final Map brokerMap; + private final Map> brokerToReplicaMap; public ClusterModelSnapshot() { rackToBrokerMap = new HashMap<>(); @@ -42,7 +40,7 @@ public ClusterModelSnapshot() { public void aggregate() { // Override broker load with sum of replicas - for (Map.Entry> entry : brokerToReplicaMap.entrySet()) { + for (Map.Entry> entry : brokerToReplicaMap.entrySet()) { int brokerId = entry.getKey(); for (Resource resource : Resource.cachedValues()) { double sum = entry.getValue().values().stream().mapToDouble(e -> e.load(resource)).sum(); @@ -51,10 +49,10 @@ public void aggregate() { } } - public void addBroker(String rack, Broker broker) { - rackToBrokerMap.putIfAbsent(rack, broker.getBrokerId()); - brokerMap.putIfAbsent(broker.getBrokerId(), broker); - brokerToReplicaMap.putIfAbsent(broker.getBrokerId(), new HashMap<>()); + public void addBroker(int brokerId, String rack, BrokerUpdater.Broker broker) { + rackToBrokerMap.putIfAbsent(rack, brokerId); + brokerMap.putIfAbsent(brokerId, broker); + brokerToReplicaMap.putIfAbsent(brokerId, new HashMap<>()); } public void removeBroker(String rack, int brokerId) { @@ -63,20 +61,20 @@ public void removeBroker(String rack, int brokerId) { brokerToReplicaMap.remove(brokerId); } - public void addTopicPartition(int brokerId, TopicPartitionReplica replica) { + public void addTopicPartition(int brokerId, TopicPartition tp, TopicPartitionReplicaUpdater.TopicPartitionReplica tpInstance) { brokerToReplicaMap.putIfAbsent(brokerId, new HashMap<>()); - brokerToReplicaMap.get(brokerId).put(replica.getTopicPartition(), replica); + brokerToReplicaMap.get(brokerId).put(tp, tpInstance); } - public Broker broker(int brokerId) { + public BrokerUpdater.Broker broker(int brokerId) { return brokerMap.get(brokerId); } - public Collection brokers() { + public Collection brokers() { return brokerMap.values(); } - public TopicPartitionReplica replica(int brokerId, TopicPartition tp) { + public TopicPartitionReplicaUpdater.TopicPartitionReplica replica(int brokerId, TopicPartition tp) { if (!brokerToReplicaMap.containsKey(brokerId)) { return null; } @@ -86,22 +84,22 @@ public TopicPartitionReplica replica(int brokerId, TopicPartition tp) { return brokerToReplicaMap.get(brokerId).get(tp); } - public Collection replicasFor(int brokerId) { + public Collection replicasFor(int brokerId) { return brokerToReplicaMap.get(brokerId).values(); } public void applyAction(Action action) { - Broker srcBroker = brokerMap.get(action.getSrcBrokerId()); - Broker destBroker = brokerMap.get(action.getDestBrokerId()); + BrokerUpdater.Broker srcBroker = brokerMap.get(action.getSrcBrokerId()); + BrokerUpdater.Broker destBroker = brokerMap.get(action.getDestBrokerId()); if (srcBroker == null || destBroker == null) { return; } - TopicPartitionReplica srcReplica = brokerToReplicaMap.get(action.getSrcBrokerId()).get(action.getSrcTopicPartition()); + TopicPartitionReplicaUpdater.TopicPartitionReplica srcReplica = brokerToReplicaMap.get(action.getSrcBrokerId()).get(action.getSrcTopicPartition()); ModelUtils.moveReplicaLoad(srcBroker, destBroker, srcReplica); brokerToReplicaMap.get(action.getSrcBrokerId()).remove(action.getSrcTopicPartition()); brokerToReplicaMap.get(action.getDestBrokerId()).put(action.getSrcTopicPartition(), srcReplica); if (action.getType() == ActionType.SWAP) { - TopicPartitionReplica destReplica = brokerToReplicaMap.get(action.getDestBrokerId()).get(action.getDestTopicPartition()); + TopicPartitionReplicaUpdater.TopicPartitionReplica destReplica = brokerToReplicaMap.get(action.getDestBrokerId()).get(action.getDestTopicPartition()); ModelUtils.moveReplicaLoad(destBroker, srcBroker, destReplica); brokerToReplicaMap.get(action.getDestBrokerId()).remove(action.getDestTopicPartition()); brokerToReplicaMap.get(action.getSrcBrokerId()).put(action.getDestTopicPartition(), destReplica); diff --git a/core/src/main/java/kafka/autobalancer/model/ModelUtils.java b/core/src/main/java/kafka/autobalancer/model/ModelUtils.java index 6197b9816f..ecab86eec1 100644 --- a/core/src/main/java/kafka/autobalancer/model/ModelUtils.java +++ b/core/src/main/java/kafka/autobalancer/model/ModelUtils.java @@ -21,11 +21,12 @@ public class ModelUtils { - public static void moveReplicaLoad(BrokerUpdater.Broker src, BrokerUpdater.Broker dest, TopicPartitionReplicaUpdater.TopicPartitionReplica replica) { + public static void moveReplicaLoad(BrokerUpdater.Broker src, BrokerUpdater.Broker dest, + TopicPartitionReplicaUpdater.TopicPartitionReplica replica) { for (Resource resource : replica.getResources()) { double delta = replica.load(resource); - src.reduceLoad(resource, delta); - dest.addLoad(resource, delta); + src.setLoad(resource, src.load(resource) - delta); + dest.setLoad(resource, dest.load(resource) + delta); } } diff --git a/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java b/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java index e6b5c5546b..f1ae0dcd1a 100644 --- a/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java +++ b/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java @@ -20,66 +20,67 @@ import kafka.autobalancer.common.RawMetricType; import kafka.autobalancer.common.Resource; import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -public class TopicPartitionReplicaUpdater { - private static final Logger LOGGER = LoggerFactory.getLogger(TopicPartitionReplicaUpdater.class); - private final Lock lock = new ReentrantLock(); +public class TopicPartitionReplicaUpdater extends AbstractInstanceUpdater { + private static final List MANDATORY_METRICS = RawMetricType.partitionMetricTypes(); private final TopicPartitionReplica replica; public TopicPartitionReplicaUpdater(TopicPartition tp) { this.replica = new TopicPartitionReplica(tp); } - public static class TopicPartitionReplica { + public TopicPartition topicPartition() { + return this.replica.getTopicPartition(); + } + + @Override + protected boolean validateMetrics(Map metricsMap) { + return metricsMap.keySet().containsAll(MANDATORY_METRICS); + } + + @Override + protected AbstractInstance instance() { + return replica; + } + + @Override + protected boolean isValidInstance() { + return true; + } + + public static class TopicPartitionReplica extends AbstractInstance { private final TopicPartition tp; - private final double[] loads = new double[Resource.cachedValues().size()]; - private final Map metricsMap = new HashMap<>(); - private final Set resources = new HashSet<>(); - private long timestamp; public TopicPartitionReplica(TopicPartition tp) { this.tp = tp; } public TopicPartitionReplica(TopicPartitionReplica other) { + super(other); this.tp = new TopicPartition(other.tp.topic(), other.tp.partition()); - System.arraycopy(other.loads, 0, this.loads, 0, other.loads.length); - this.resources.addAll(other.resources); - this.timestamp = other.timestamp; - } - - public Set getResources() { - return this.resources; } - public void setLoad(Resource resource, double value) { - this.resources.add(resource); - this.loads[resource.id()] = value; - } - - public double load(Resource resource) { - if (!this.resources.contains(resource)) { - return 0.0; + @Override + public void processMetrics() { + for (Map.Entry entry : metricsMap.entrySet()) { + if (entry.getKey().metricScope() != RawMetricType.MetricScope.PARTITION) { + continue; + } + switch (entry.getKey()) { + case TOPIC_PARTITION_BYTES_IN: + this.setLoad(Resource.NW_IN, entry.getValue()); + break; + case TOPIC_PARTITION_BYTES_OUT: + this.setLoad(Resource.NW_OUT, entry.getValue()); + break; + default: + break; + } } - return this.loads[resource.id()]; - } - - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; - } - - public long getTimestamp() { - return timestamp; } public TopicPartition getTopicPartition() { @@ -99,102 +100,29 @@ public int hashCode() { return Objects.hashCode(tp); } - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append("{TopicPartition=") - .append(tp) - .append(", timestamp=").append(timestamp) - .append(", Loads=["); - for (int i = 0; i < loads.length; i++) { - builder.append(Resource.of(i).resourceString(loads[i])); - if (i != loads.length - 1) { - builder.append(", "); - } - } - builder.append("]"); - int i = 0; - for (Map.Entry entry : metricsMap.entrySet()) { - if (i == 0) { - builder.append(" Metrics={"); - } - builder.append(entry.getKey()) - .append("=") - .append(entry.getValue()); - if (i != metricsMap.size() - 1) { - builder.append(", "); - } - i++; - } - builder.append("}"); - return builder.toString(); - } - } - - public boolean update(Map metricsMap, long time) { - if (!metricsMap.keySet().containsAll(RawMetricType.partitionMetricTypes())) { - LOGGER.error("Topic partition {} metrics sanity check failed, metrics is incomplete {}", replica.getTopicPartition(), metricsMap.keySet()); - return false; + public String shortString() { + return "TopicPartitionReplica{" + + "tp=" + tp + + ", " + timeString() + + ", " + loadString() + + "}"; } - lock.lock(); - try { - if (time < this.replica.getTimestamp()) { - LOGGER.warn("Outdated topic partition {} metrics at time {}, last updated time {}", replica.getTopicPartition(), time, this.replica.getTimestamp()); - return false; - } - for (Map.Entry entry : metricsMap.entrySet()) { - if (entry.getKey().metricScope() != RawMetricType.MetricScope.PARTITION) { - continue; - } - switch (entry.getKey()) { - case TOPIC_PARTITION_BYTES_IN: - this.replica.setLoad(Resource.NW_IN, entry.getValue()); - break; - case TOPIC_PARTITION_BYTES_OUT: - this.replica.setLoad(Resource.NW_OUT, entry.getValue()); - break; - default: - metricsMap.put(entry.getKey(), entry.getValue()); - break; - } - } - this.replica.setTimestamp(time); - } finally { - lock.unlock(); + @Override + public AbstractInstance copy() { + return new TopicPartitionReplica(this); } - LOGGER.debug("Successfully updated on {} at time {}", this.replica.getTopicPartition(), this.replica.getTimestamp()); - return true; - } - public TopicPartitionReplica get() { - TopicPartitionReplica replica; - lock.lock(); - try { - replica = new TopicPartitionReplica(this.replica); - } finally { - lock.unlock(); + @Override + protected String name() { + return tp.toString(); } - return replica; - } - public TopicPartitionReplica get(long timeSince) { - TopicPartitionReplica replica; - lock.lock(); - try { - if (this.replica.timestamp < timeSince) { - LOGGER.debug("Topic partition {} metrics is out of sync, expected earliest time: {}, actual: {}", - this.replica.getTopicPartition(), timeSince, this.replica.timestamp); - return null; - } - replica = new TopicPartitionReplica(this.replica); - } finally { - lock.unlock(); + @Override + public String toString() { + return "{TopicPartition=" + tp + + ", " + super.toString() + + "}"; } - return replica; - } - - public TopicPartition topicPartition() { - return this.replica.getTopicPartition(); } } diff --git a/core/src/main/resources/jmx/rules/broker.yaml b/core/src/main/resources/jmx/rules/broker.yaml index 1b17d2c8c0..526604d6da 100644 --- a/core/src/main/resources/jmx/rules/broker.yaml +++ b/core/src/main/resources/jmx/rules/broker.yaml @@ -110,6 +110,10 @@ rules: metric: kafka.request.time.99p type: gauge desc: The 99th percentile time the broker has taken to service requests + Max: + metric: kafka.request.time.max + type: gauge + desc: The max time the broker has taken to service requests - bean: kafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request=* metricAttribute: @@ -132,10 +136,6 @@ rules: metric: kafka.request.queue.time.99p type: gauge desc: The 99th percentile time the broker has taken to dequeue requests - Max: - metric: kafka.request.time.max - type: gauge - desc: The max time the broker has taken to service requests - bean: kafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request=* metricAttribute: diff --git a/core/src/main/scala/kafka/log/stream/s3/telemetry/TelemetryManager.java b/core/src/main/scala/kafka/log/stream/s3/telemetry/TelemetryManager.java index a3a46e65af..5cbe3d83d1 100644 --- a/core/src/main/scala/kafka/log/stream/s3/telemetry/TelemetryManager.java +++ b/core/src/main/scala/kafka/log/stream/s3/telemetry/TelemetryManager.java @@ -78,6 +78,7 @@ public class TelemetryManager { private final String clusterId; private final List metricReaderList; private final List autoCloseables; + private JmxMetricInsight jmxMetricInsight; private PrometheusHttpServer prometheusHttpServer; public TelemetryManager(KafkaConfig kafkaConfig, String clusterId) { @@ -149,7 +150,7 @@ public static OpenTelemetrySdk getOpenTelemetrySdk() { } private void addJmxMetrics(OpenTelemetry ot) { - JmxMetricInsight jmxMetricInsight = JmxMetricInsight.createService(ot, kafkaConfig.s3ExporterReportIntervalMs()); + jmxMetricInsight = JmxMetricInsight.createService(ot, kafkaConfig.s3ExporterReportIntervalMs()); MetricConfiguration conf = new MetricConfiguration(); Set roles = kafkaConfig.processRoles(); diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 8cc53e7c28..ffa267486c 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -18,7 +18,7 @@ package kafka.server import com.automq.stream.s3.metadata.ObjectUtils -import kafka.autobalancer.AutoBalancerManager +import kafka.autobalancer.{AutoBalancerManager, AutoBalancerService} import kafka.autobalancer.config.AutoBalancerControllerConfig import kafka.cluster.Broker.ServerInfo import kafka.log.stream.s3.ConfigUtils @@ -108,9 +108,9 @@ class ControllerServer( var controllerApis: ControllerApis = _ var controllerApisHandlerPool: KafkaRequestHandlerPool = _ var migrationSupport: Option[ControllerMigrationSupport] = None - var autoBalancerManager: AutoBalancerManager = _ + var autoBalancerManager: Option[AutoBalancerService] = None - def buildAutoBalancerManager: AutoBalancerManager = { + protected def buildAutoBalancerManager: AutoBalancerService = { new AutoBalancerManager(time, config, controller, raftManager.client) } @@ -304,10 +304,12 @@ class ControllerServer( s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent", DataPlaneAcceptor.ThreadPrefix) - if (config.getBoolean(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ENABLE)) { - autoBalancerManager = buildAutoBalancerManager - autoBalancerManager.start() + autoBalancerManager = if (config.getBoolean(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ENABLE)) { + Some(buildAutoBalancerManager) + } else { + None } + autoBalancerManager.foreach(_.start()) /** * Enable the controller endpoint(s). If we are using an authorizer which stores * ACLs in the metadata log, such as StandardAuthorizer, we will be able to start @@ -334,8 +336,7 @@ class ControllerServer( // Ensure that we're not the Raft leader prior to shutting down our socket server, for a // smoother transition. sharedServer.ensureNotRaftLeader() - if (autoBalancerManager != null) - CoreUtils.swallow(autoBalancerManager.shutdown(), logging = this) + autoBalancerManager.foreach(_.shutdown()) if (socketServer != null) CoreUtils.swallow(socketServer.stopProcessingRequests(), this) migrationSupport.foreach(_.shutdown(this)) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index c85f5b8388..4167dc877c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -240,8 +240,6 @@ class ReplicaManager(val config: KafkaConfig, val delayedElectLeaderPurgatory = delayedElectLeaderPurgatoryParam.getOrElse( DelayedOperationPurgatory[DelayedElectLeader]( purgatoryName = "ElectLeader", brokerId = config.brokerId)) - // This threadPool is used to separate slow fetches from quick fetches. - val slowFetchExecutors = Executors.newFixedThreadPool(4, ThreadUtils.createThreadFactory("slow-fetch-executor-%d", true)) /* epoch of the controller that last changed the leader */ @volatile private[server] var controllerEpoch: Int = KafkaController.InitialControllerEpoch @@ -2439,7 +2437,6 @@ class ReplicaManager(val config: KafkaConfig, replicaSelectorOpt.foreach(_.close) removeAllTopicMetrics() - slowFetchExecutors.shutdown() info("Shut down completely") } diff --git a/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java b/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java index 196971eeb4..21c4c11b18 100644 --- a/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java +++ b/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java @@ -20,10 +20,8 @@ import kafka.autobalancer.config.AutoBalancerConfig; import kafka.autobalancer.config.AutoBalancerControllerConfig; import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; -import kafka.autobalancer.goals.NetworkInCapacityGoal; -import kafka.autobalancer.goals.NetworkInDistributionGoal; -import kafka.autobalancer.goals.NetworkOutCapacityGoal; -import kafka.autobalancer.goals.NetworkOutDistributionGoal; +import kafka.autobalancer.goals.NetworkInUsageDistributionGoal; +import kafka.autobalancer.goals.NetworkOutUsageDistributionGoal; import kafka.autobalancer.metricsreporter.AutoBalancerMetricsReporter; import kafka.autobalancer.utils.AbstractLoadGenerator; import kafka.autobalancer.utils.AutoBalancerClientsIntegrationTestHarness; @@ -87,8 +85,6 @@ public Map overridingBrokerProps() { Map props = new HashMap<>(); props.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, AutoBalancerMetricsReporter.class.getName()); props.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_CONFIG, "1000"); - props.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_IN_CAPACITY, "100"); // KB/s - props.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, "100"); // KB/s return props; } @@ -97,12 +93,10 @@ public Map overridingBrokerProps() { public Map overridingControllerProps() { Map props = new HashMap<>(); props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, new StringJoiner(",") - .add(NetworkInCapacityGoal.class.getName()) - .add(NetworkOutCapacityGoal.class.getName()) - .add(NetworkInDistributionGoal.class.getName()) - .add(NetworkOutDistributionGoal.class.getName()).toString()); - props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD, "0.2"); - props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD, "0.2"); + .add(NetworkInUsageDistributionGoal.class.getName()) + .add(NetworkOutUsageDistributionGoal.class.getName()).toString()); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD, "0"); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD, "0"); props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION, "0.2"); props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION, "0.2"); props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS, "10000"); diff --git a/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java b/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java index 331bddd8b9..b16970e60a 100644 --- a/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java +++ b/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java @@ -25,7 +25,7 @@ import kafka.autobalancer.model.ClusterModel; import kafka.autobalancer.model.ClusterModelSnapshot; import kafka.autobalancer.model.RecordClusterModel; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; import kafka.autobalancer.utils.AutoBalancerClientsIntegrationTestHarness; import kafka.cluster.EndPoint; import kafka.server.BrokerServer; @@ -88,14 +88,14 @@ public Map overridingBrokerProps() { } private boolean checkConsumeRecord(ClusterModel clusterModel, int brokerId, long delay) { - ClusterModelSnapshot snapshot = clusterModel.snapshot(Collections.emptySet(), Collections.emptySet(), delay, true); + ClusterModelSnapshot snapshot = clusterModel.snapshot(Collections.emptySet(), Collections.emptySet(), delay); if (snapshot.broker(brokerId) == null) { return false; } TopicPartition testTp = new TopicPartition(TOPIC_0, 0); TopicPartition metricTp = new TopicPartition(METRIC_TOPIC, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica testReplica = snapshot.replica(brokerId, testTp); - TopicPartitionReplicaUpdater.TopicPartitionReplica metricReplica = snapshot.replica(brokerId, metricTp); + TopicPartitionReplica testReplica = snapshot.replica(brokerId, testTp); + TopicPartitionReplica metricReplica = snapshot.replica(brokerId, metricTp); if (testReplica == null || metricReplica == null) { return false; } @@ -186,13 +186,13 @@ public void testConsume() throws InterruptedException { loadRetriever.onBrokerUnregister(unregisterRecord); Thread.sleep(5000); Assertions.assertTrue(() -> { - ClusterModelSnapshot snapshot = clusterModel.snapshot(Collections.emptySet(), Collections.emptySet(), 3000L, true); + ClusterModelSnapshot snapshot = clusterModel.snapshot(Collections.emptySet(), Collections.emptySet(), 3000L); if (snapshot.broker(brokerConfig.brokerId()) != null) { return false; } - TopicPartitionReplicaUpdater.TopicPartitionReplica testReplica = snapshot.replica(brokerConfig.brokerId(), + TopicPartitionReplica testReplica = snapshot.replica(brokerConfig.brokerId(), new TopicPartition(TOPIC_0, 0)); - TopicPartitionReplicaUpdater.TopicPartitionReplica metricReplica = snapshot.replica(brokerConfig.brokerId(), + TopicPartitionReplica metricReplica = snapshot.replica(brokerConfig.brokerId(), new TopicPartition(METRIC_TOPIC, 0)); return testReplica == null && metricReplica == null; }); diff --git a/core/src/test/java/kafka/autobalancer/common/normalizer/NormalizerTest.java b/core/src/test/java/kafka/autobalancer/common/normalizer/NormalizerTest.java new file mode 100644 index 0000000000..cc6d9ab431 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/common/normalizer/NormalizerTest.java @@ -0,0 +1,55 @@ +/* + * 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 kafka.autobalancer.common.normalizer; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class NormalizerTest { + private static final double EPSILON = 0.0001; + + @Test + public void testLinearNormalize() { + Normalizer normalizer = new LinearNormalizer(0, 100); + Assertions.assertEquals(-0.1, normalizer.normalize(-10), EPSILON); + Assertions.assertEquals(0.0, normalizer.normalize(0), EPSILON); + Assertions.assertEquals(0.4, normalizer.normalize(40), EPSILON); + Assertions.assertEquals(1.0, normalizer.normalize(100), EPSILON); + Assertions.assertEquals(1.2, normalizer.normalize(120), EPSILON); + + Assertions.assertEquals(1.1, normalizer.normalize(-10, true), EPSILON); + Assertions.assertEquals(1.0, normalizer.normalize(0, true), EPSILON); + Assertions.assertEquals(0.6, normalizer.normalize(40, true), EPSILON); + Assertions.assertEquals(0.0, normalizer.normalize(100, true), EPSILON); + Assertions.assertEquals(-0.2, normalizer.normalize(120, true), EPSILON); + } + + @Test + public void testStepNormalizer() { + Normalizer normalizer = new StepNormalizer(0, 100, 0.9); + Assertions.assertEquals(-0.09, normalizer.normalize(-10), EPSILON); + Assertions.assertEquals(0, normalizer.normalize(0), EPSILON); + Assertions.assertEquals(0.36, normalizer.normalize(40), EPSILON); + Assertions.assertEquals(0.9, normalizer.normalize(100), EPSILON); + double v1 = normalizer.normalize(120); + double v2 = normalizer.normalize(Double.MAX_VALUE); + Assertions.assertTrue(v1 > 0.9 && v1 < 1.0); + Assertions.assertTrue(v2 > 0.9 && v2 < 1.0); + Assertions.assertTrue(v1 < v2); + } +} diff --git a/core/src/test/java/kafka/autobalancer/detector/AnomalyDetectorTest.java b/core/src/test/java/kafka/autobalancer/detector/AnomalyDetectorTest.java index 7a3f11e98a..c4ee4aa9a6 100644 --- a/core/src/test/java/kafka/autobalancer/detector/AnomalyDetectorTest.java +++ b/core/src/test/java/kafka/autobalancer/detector/AnomalyDetectorTest.java @@ -20,19 +20,21 @@ import com.automq.stream.s3.metrics.TimerUtil; import kafka.autobalancer.common.Action; import kafka.autobalancer.common.RawMetricType; +import kafka.autobalancer.common.Resource; import kafka.autobalancer.config.AutoBalancerControllerConfig; import kafka.autobalancer.executor.ActionExecutorService; import kafka.autobalancer.goals.Goal; -import kafka.autobalancer.goals.NetworkInDistributionGoal; -import kafka.autobalancer.goals.NetworkOutDistributionGoal; +import kafka.autobalancer.goals.NetworkInUsageDistributionGoal; +import kafka.autobalancer.goals.NetworkOutUsageDistributionGoal; import kafka.autobalancer.model.ClusterModel; +import kafka.autobalancer.model.ClusterModelSnapshot; import org.apache.commons.math3.distribution.PoissonDistribution; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -66,7 +68,6 @@ public void testSchedulingTimeCost() { for (int j = 0; j < partitionNumPerTopic; j++) { clusterModel.createPartition(topicId, j, brokerIndex); Map metrics = generateRandomMetrics(r); - clusterModel.updateBrokerMetrics(brokerIndex, metrics, System.currentTimeMillis()); clusterModel.updateTopicPartitionMetrics(brokerIndex, new TopicPartition(topicName, j), metrics, System.currentTimeMillis()); currPartitionNum++; if (currPartitionNum >= partitionNums[brokerIndex]) { @@ -77,11 +78,12 @@ public void testSchedulingTimeCost() { } Map configs = new AutoBalancerControllerConfig(Collections.emptyMap(), false).originals(); - Goal goal0 = new NetworkInDistributionGoal(); - Goal goal1 = new NetworkOutDistributionGoal(); + Goal goal0 = new NetworkInUsageDistributionGoal(); + Goal goal1 = new NetworkOutUsageDistributionGoal(); goal0.configure(configs); goal1.configure(configs); + List actionList = new ArrayList<>(); AnomalyDetector detector = new AnomalyDetectorBuilder() .clusterModel(clusterModel) .addGoal(goal0) @@ -99,22 +101,42 @@ public void shutdown() { @Override public void execute(Action action) { - + actionList.add(action); } @Override public void execute(List actions) { - + actionList.addAll(actions); } }) .build(); - Assertions.assertTimeout(Duration.ofMillis(1000), () -> { - TimerUtil timerUtil = new TimerUtil(); - detector.resume(); - detector.detect(); - System.out.printf("Detect cost: %d ms%n", timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); - }); + TimerUtil timerUtil = new TimerUtil(); + detector.resume(); + detector.detect(); + System.out.printf("Detect cost: %d ms, %d actions detected%n", timerUtil.elapsedAs(TimeUnit.MILLISECONDS), actionList.size()); + Assertions.assertFalse(actionList.isEmpty()); + + ClusterModelSnapshot snapshot = clusterModel.snapshot(); + + double[] loadsBefore = snapshot.brokers().stream().map(b -> b.load(Resource.NW_IN)).mapToDouble(Double::doubleValue).toArray(); + double meanBefore = Arrays.stream(loadsBefore).sum() / loadsBefore.length; + double stdDevBefore = calculateStdDev(meanBefore, loadsBefore); + for (Action action : actionList) { + snapshot.applyAction(action); + } + double[] loadsAfter = snapshot.brokers().stream().map(b -> b.load(Resource.NW_IN)).mapToDouble(Double::doubleValue).toArray(); + double meanAfter = Arrays.stream(loadsBefore).sum() / loadsBefore.length; + double stdDevAfter = calculateStdDev(meanAfter, loadsAfter); + Assertions.assertEquals(meanBefore, meanAfter); + Assertions.assertTrue(stdDevAfter < stdDevBefore); + System.out.printf("mean: %f, stdDev before: %f (%.2f%%), after: %f (%.2f%%), %n", meanAfter, stdDevBefore, + 100.0 * stdDevBefore / meanBefore, stdDevAfter, 100.0 * stdDevAfter / meanAfter); + } + + private double calculateStdDev(double mean, double[] values) { + double sum = Arrays.stream(values).map(v -> Math.pow(v - mean, 2)).sum(); + return Math.sqrt(sum / values.length); } private int[] generatePartitionDist(int totalPartitionNum, int brokerNum) { @@ -132,11 +154,6 @@ private int[] generatePartitionDist(int totalPartitionNum, int brokerNum) { private Map generateRandomMetrics(Random r) { Map metrics = new HashMap<>(); - metrics.put(RawMetricType.BROKER_CAPACITY_NW_IN, 20.0 * 1024 * 1024); - metrics.put(RawMetricType.BROKER_CAPACITY_NW_OUT, 20.0 * 1024 * 1024); - metrics.put(RawMetricType.ALL_TOPIC_BYTES_IN, 20.0 * 1024 * 1024); - metrics.put(RawMetricType.ALL_TOPIC_BYTES_OUT, 20.0 * 1024 * 1024); - metrics.put(RawMetricType.BROKER_CPU_UTIL, 0.0); metrics.put(RawMetricType.TOPIC_PARTITION_BYTES_OUT, r.nextDouble(0, 1024 * 1024)); metrics.put(RawMetricType.TOPIC_PARTITION_BYTES_IN, r.nextDouble(0, 1024 * 1024)); metrics.put(RawMetricType.PARTITION_SIZE, 0.0); diff --git a/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java b/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java index 6f7b739220..a54df32ca0 100644 --- a/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java +++ b/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java @@ -21,9 +21,9 @@ import kafka.autobalancer.common.ActionType; import kafka.autobalancer.common.Resource; import kafka.autobalancer.config.AutoBalancerControllerConfig; -import kafka.autobalancer.model.BrokerUpdater; +import kafka.autobalancer.model.BrokerUpdater.Broker; import kafka.autobalancer.model.ClusterModelSnapshot; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -34,6 +34,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.StringJoiner; @Tag("S3Unit") @@ -45,14 +46,10 @@ public class AbstractGoalTest extends GoalTestBase { public void setup() { Map config = new HashMap<>(); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, new StringJoiner(",") - .add(NetworkInDistributionGoal.class.getName()) - .add(NetworkOutDistributionGoal.class.getName()) - .add(NetworkInCapacityGoal.class.getName()) - .add(NetworkOutCapacityGoal.class.getName()).toString()); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_UTILIZATION_THRESHOLD, 0.8); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_UTILIZATION_THRESHOLD, 0.8); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD, 0.2); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD, 0.2); + .add(NetworkInUsageDistributionGoal.class.getName()) + .add(NetworkOutUsageDistributionGoal.class.getName()).toString()); + config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD, 0); + config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD, 0); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION, 0.2); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION, 0.2); AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(config, false); @@ -66,160 +63,49 @@ public void setup() { @Test public void testCalculateAcceptanceScore() { ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - BrokerUpdater.Broker broker2 = createBroker(cluster, RACK, 2, true); - - broker0.setCapacity(Resource.NW_IN, 50); - broker0.setLoad(Resource.NW_IN, 40); - - broker1.setCapacity(Resource.NW_IN, 50); - broker1.setLoad(Resource.NW_IN, 80); - - broker2.setCapacity(Resource.NW_IN, 50); - broker2.setLoad(Resource.NW_IN, 120); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - replica0.setLoad(Resource.NW_IN, 40); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 1, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 2); - replica1.setLoad(Resource.NW_IN, 40); - replica2.setLoad(Resource.NW_IN, 40); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 2, TOPIC_0, 3); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 2, TOPIC_0, 4); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 2, TOPIC_0, 5); - replica3.setLoad(Resource.NW_IN, 40); - replica4.setLoad(Resource.NW_IN, 40); - replica5.setLoad(Resource.NW_IN, 40); - - Goal capacityGoal = goalMap.get(NetworkInCapacityGoal.class.getSimpleName()); - - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 1), cluster)); - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 2), cluster)); - - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 0), cluster)); - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 2), cluster)); - - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 0), cluster)); - Assertions.assertEquals(0, capacityGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 1), cluster)); - - Goal distributionGoal = goalMap.get(NetworkInDistributionGoal.class.getSimpleName()); - - Assertions.assertEquals(0.1, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 1), cluster), 0.001); - Assertions.assertEquals(0.1, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 2), cluster), 0.001); - - Assertions.assertEquals(0.5, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 0), cluster), 0.001); - Assertions.assertEquals(0.1, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 2), cluster), 0.001); - - Assertions.assertEquals(0.9, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 0), cluster), 0.001); - Assertions.assertEquals(0.5, distributionGoal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 1), cluster), 0.001); - - for (Goal goal : goalMap.values()) { - List actions = goal.optimize(cluster, goalMap.values()); - Assertions.assertTrue(actions.isEmpty()); - } - } - - @Test - public void testMultiGoalOptimization() { - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - BrokerUpdater.Broker broker2 = createBroker(cluster, RACK, 2, true); - BrokerUpdater.Broker broker3 = createBroker(cluster, RACK, 3, true); - - broker0.setCapacity(Resource.NW_IN, 100); - broker0.setCapacity(Resource.NW_OUT, 100); - broker0.setLoad(Resource.NW_IN, 90); - broker0.setLoad(Resource.NW_OUT, 50); - - broker1.setCapacity(Resource.NW_IN, 100); - broker1.setCapacity(Resource.NW_OUT, 100); - broker1.setLoad(Resource.NW_IN, 20); - broker1.setLoad(Resource.NW_OUT, 90); - - broker2.setCapacity(Resource.NW_IN, 100); - broker2.setCapacity(Resource.NW_OUT, 100); - broker2.setLoad(Resource.NW_IN, 30); - broker2.setLoad(Resource.NW_OUT, 70); - - broker3.setCapacity(Resource.NW_IN, 100); - broker3.setCapacity(Resource.NW_OUT, 100); - broker3.setLoad(Resource.NW_IN, 60); - broker3.setLoad(Resource.NW_OUT, 10); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_1, 1); - replica0.setLoad(Resource.NW_IN, 40); - replica0.setLoad(Resource.NW_OUT, 30); - replica1.setLoad(Resource.NW_IN, 30); - replica1.setLoad(Resource.NW_OUT, 15); - replica2.setLoad(Resource.NW_IN, 20); - replica2.setLoad(Resource.NW_OUT, 5); - Assertions.assertEquals(90, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(50, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_1, 2); - replica3.setLoad(Resource.NW_IN, 5); - replica3.setLoad(Resource.NW_OUT, 50); - replica4.setLoad(Resource.NW_IN, 15); - replica4.setLoad(Resource.NW_OUT, 40); - Assertions.assertEquals(20, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(90, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 2, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 2, TOPIC_2, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 2, TOPIC_3, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 2, TOPIC_0, 1); - replica5.setLoad(Resource.NW_IN, 10); - replica5.setLoad(Resource.NW_OUT, 10); - replica6.setLoad(Resource.NW_IN, 2); - replica6.setLoad(Resource.NW_OUT, 30); - replica7.setLoad(Resource.NW_IN, 3); - replica7.setLoad(Resource.NW_OUT, 15); - replica8.setLoad(Resource.NW_IN, 15); - replica8.setLoad(Resource.NW_OUT, 15); - Assertions.assertEquals(30, cluster.replicasFor(2).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(70, cluster.replicasFor(2).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica9 = createTopicPartition(cluster, 3, TOPIC_0, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica10 = createTopicPartition(cluster, 3, TOPIC_2, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica11 = createTopicPartition(cluster, 3, TOPIC_3, 1); - replica9.setLoad(Resource.NW_IN, 40); - replica9.setLoad(Resource.NW_OUT, 1); - replica10.setLoad(Resource.NW_IN, 8); - replica10.setLoad(Resource.NW_OUT, 4); - replica11.setLoad(Resource.NW_IN, 12); - replica11.setLoad(Resource.NW_OUT, 5); - Assertions.assertEquals(60, cluster.replicasFor(3).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(10, cluster.replicasFor(3).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - for (Goal goal : goalMap.values()) { - goal.optimize(cluster, goalMap.values()); - } - for (BrokerUpdater.Broker broker : cluster.brokers()) { - Assertions.assertTrue(goalMap.get(NetworkInCapacityGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - Assertions.assertTrue(goalMap.get(NetworkOutCapacityGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - Assertions.assertTrue(goalMap.get(NetworkInDistributionGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - if (broker.getBrokerId() == 2) { - Assertions.assertFalse(goalMap.get(NetworkOutDistributionGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - } else { - Assertions.assertTrue(goalMap.get(NetworkOutDistributionGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - } - } - for (Goal goal : goalMap.values()) { - goal.optimize(cluster, goalMap.values()); - } - // all goals succeed in second iteration - for (BrokerUpdater.Broker broker : cluster.brokers()) { - Assertions.assertTrue(goalMap.get(NetworkInCapacityGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - Assertions.assertTrue(goalMap.get(NetworkOutCapacityGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - Assertions.assertTrue(goalMap.get(NetworkInDistributionGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - Assertions.assertTrue(goalMap.get(NetworkOutDistributionGoal.class.getSimpleName()).isBrokerAcceptable(broker)); - } + Broker broker0 = createBroker(cluster, RACK, 0, true); + Broker broker1 = createBroker(cluster, RACK, 1, true); + Broker broker2 = createBroker(cluster, RACK, 2, true); + + broker0.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + broker1.setLoad(Resource.NW_IN, 80 * 1024 * 1024); + broker2.setLoad(Resource.NW_IN, 120 * 1024 * 1024); + + TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); + replica0.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + + TopicPartitionReplica replica1 = createTopicPartition(cluster, 1, TOPIC_0, 1); + TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 2); + replica1.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + replica2.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + + TopicPartitionReplica replica3 = createTopicPartition(cluster, 2, TOPIC_0, 3); + TopicPartitionReplica replica4 = createTopicPartition(cluster, 2, TOPIC_0, 4); + TopicPartitionReplica replica5 = createTopicPartition(cluster, 2, TOPIC_0, 5); + replica3.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + replica4.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + replica5.setLoad(Resource.NW_IN, 40 * 1024 * 1024); + + Goal goal = goalMap.get(NetworkInUsageDistributionGoal.class.getSimpleName()); + AbstractResourceDistributionGoal distributionGoal = (AbstractResourceDistributionGoal) goal; + distributionGoal.calculateResourceBound(Set.of(broker0, broker1, broker2)); + + Assertions.assertEquals(0.3, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 1), cluster), 0.001); + Assertions.assertEquals(0.3, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 0), 0, 2), cluster), 0.001); + + Assertions.assertEquals(0.5, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 0), cluster), 0.001); + Assertions.assertEquals(0.3, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 2), 1, 2), cluster), 0.001); + + Assertions.assertEquals(0.7, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 0), cluster), 0.001); + Assertions.assertEquals(0.5, goal.actionAcceptanceScore(new Action(ActionType.MOVE, new TopicPartition(TOPIC_0, 4), 2, 1), cluster), 0.001); + + List actions = goal.optimize(cluster, List.of(goal)); + Assertions.assertFalse(actions.isEmpty()); + Assertions.assertEquals(1, actions.size()); + Assertions.assertEquals(2, actions.get(0).getSrcBrokerId()); + Assertions.assertEquals(0, actions.get(0).getDestBrokerId()); + Assertions.assertEquals(TOPIC_0, actions.get(0).getSrcTopicPartition().topic()); + Assertions.assertTrue(Set.of(3, 4, 5).contains(actions.get(0).getSrcTopicPartition().partition())); + Assertions.assertNull(actions.get(0).getDestTopicPartition()); } } diff --git a/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java deleted file mode 100644 index a8fdca4698..0000000000 --- a/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java +++ /dev/null @@ -1,373 +0,0 @@ -/* - * 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 kafka.autobalancer.goals; - -import kafka.autobalancer.common.Action; -import kafka.autobalancer.common.ActionType; -import kafka.autobalancer.common.Resource; -import kafka.autobalancer.config.AutoBalancerControllerConfig; -import kafka.autobalancer.model.BrokerUpdater; -import kafka.autobalancer.model.ClusterModelSnapshot; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.StringJoiner; - -@Tag("S3Unit") -public class AbstractResourceCapacityGoalTest extends GoalTestBase { - private final Map goalMap = new HashMap<>(); - - @BeforeEach - public void setup() { - Map config = new HashMap<>(); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, new StringJoiner(",") - .add(NetworkInCapacityGoal.class.getName()) - .add(NetworkOutCapacityGoal.class.getName()).toString()); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_UTILIZATION_THRESHOLD, 0.8); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_UTILIZATION_THRESHOLD, 0.8); - AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(config, false); - List goalList = controllerConfig.getConfiguredInstances(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, AbstractGoal.class); - for (AbstractGoal goal : goalList) { - goalMap.put(goal.name(), goal); - } - } - - private Goal getGoalByResource(Resource resource) { - Goal goal = null; - switch (resource) { - case NW_IN: - goal = goalMap.get(NetworkInCapacityGoal.class.getSimpleName()); - break; - case NW_OUT: - goal = goalMap.get(NetworkOutCapacityGoal.class.getSimpleName()); - break; - default: - break; - } - return goal; - } - - private void testActionAcceptanceScore(Resource resource) { - Goal goal = getGoalByResource(resource); - Assertions.assertNotNull(goal); - - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - BrokerUpdater.Broker broker2 = createBroker(cluster, RACK, 2, true); - broker1.setCapacity(resource, 100); - broker2.setCapacity(resource, 100); - - broker1.setLoad(resource, 90); - broker2.setLoad(resource, 50); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica = createTopicPartition(cluster, 1, TOPIC_0, 0); - replica.setLoad(resource, 20); - - Action action = new Action(ActionType.MOVE, replica.getTopicPartition(), broker1.getBrokerId(), broker2.getBrokerId()); - Assertions.assertEquals(0.6, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 50); - broker2.setLoad(resource, 30); - Assertions.assertEquals(0.5, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 30); - broker2.setLoad(resource, 50); - Assertions.assertEquals(0.4, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 75); - broker2.setLoad(resource, 70); - Assertions.assertEquals(0.0, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 75); - broker2.setLoad(resource, 85); - Assertions.assertEquals(0.0, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 90); - broker2.setLoad(resource, 70); - Assertions.assertEquals(0.0, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 90); - broker2.setLoad(resource, 85); - Assertions.assertEquals(0.0, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 90); - broker2.setLoad(resource, 50); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 2, TOPIC_0, 1); - replica2.setLoad(resource, 5); - - Action action2 = new Action(ActionType.SWAP, replica.getTopicPartition(), broker1.getBrokerId(), broker2.getBrokerId(), replica2.getTopicPartition()); - Assertions.assertEquals(0.575, goal.actionAcceptanceScore(action2, cluster), 1e-15); - - replica.setLoad(resource, 10); - Assertions.assertEquals(0.525, goal.actionAcceptanceScore(action2, cluster), 1e-15); - - replica.setLoad(resource, 1); - Assertions.assertEquals(0.0, goal.actionAcceptanceScore(action2, cluster), 1e-15); - } - - private void testSingleResourceCapacityOptimizeOneMove(Resource resource) { - Goal goal = getGoalByResource(resource); - Assertions.assertNotNull(goal); - - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - BrokerUpdater.Broker broker2 = createBroker(cluster, RACK, 2, true); - BrokerUpdater.Broker broker3 = createBroker(cluster, RACK, 3, true); - BrokerUpdater.Broker broker4 = createBroker(cluster, RACK, 4, false); - - double load0 = 90; - double load1 = 60; - double load2 = 28; - double load3 = 50; - broker0.setCapacity(resource, 100); - broker0.setLoad(resource, 90); - - broker1.setCapacity(resource, 90); - broker1.setLoad(resource, 30); - - broker2.setCapacity(resource, 30); - broker2.setLoad(resource, 28); - - broker3.setCapacity(resource, 120); - broker3.setLoad(resource, 50); - - broker4.setCapacity(resource, 999); - broker4.setLoad(resource, 0.0); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_2, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_3, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 1); - replica0.setLoad(resource, 20); - replica1.setLoad(resource, 30); - replica2.setLoad(resource, 5); - replica3.setLoad(resource, 35); - Assertions.assertEquals(load0, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(resource)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_4, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_2, 1); - replica4.setLoad(resource, 10); - replica5.setLoad(resource, 50); - Assertions.assertEquals(load1, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(resource)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 2, TOPIC_1, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 2, TOPIC_2, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 2, TOPIC_4, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica9 = createTopicPartition(cluster, 2, TOPIC_3, 1); - replica6.setLoad(resource, 15); - replica7.setLoad(resource, 5); - replica8.setLoad(resource, 2); - replica9.setLoad(resource, 6); - Assertions.assertEquals(load2, cluster.replicasFor(2).stream().mapToDouble(e -> e.load(resource)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica10 = createTopicPartition(cluster, 3, TOPIC_0, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica11 = createTopicPartition(cluster, 3, TOPIC_4, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica12 = createTopicPartition(cluster, 3, TOPIC_1, 1); - replica10.setLoad(resource, 10); - replica11.setLoad(resource, 15); - replica12.setLoad(resource, 25); - Assertions.assertEquals(load3, cluster.replicasFor(3).stream().mapToDouble(e -> e.load(resource)).sum()); - - List actions = goal.optimize(cluster, goalMap.values()); - Assertions.assertNotEquals(0, actions.size()); - Assertions.assertNotNull(cluster); - Assertions.assertEquals(0, cluster.replicasFor(4).size()); - for (BrokerUpdater.Broker broker : cluster.brokers()) { - Assertions.assertTrue(goal.isBrokerAcceptable(broker)); - } - } - - private void testSingleResourceCapacityOptimizeMultiMove(Resource resource) { - Goal goal = getGoalByResource(resource); - Assertions.assertNotNull(goal); - - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - - double load0 = 90; - double load1 = 60; - broker0.setCapacity(resource, 100); - broker0.setLoad(resource, load0); - - broker1.setCapacity(resource, 100); - broker1.setLoad(resource, load1); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_0, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 3); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 0, TOPIC_0, 4); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 0, TOPIC_0, 5); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 0, TOPIC_0, 6); - replica0.setLoad(resource, 40); - replica1.setLoad(resource, 35); - replica2.setLoad(resource, 1); - replica3.setLoad(resource, 2); - replica4.setLoad(resource, 3); - replica5.setLoad(resource, 2); - replica6.setLoad(resource, 7); - Assertions.assertEquals(load0, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(resource)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_1, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_1, 1); - replica7.setLoad(resource, 20); - replica8.setLoad(resource, 40); - Assertions.assertEquals(load1, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(resource)).sum()); - - List actions = goal.optimize(cluster, goalMap.values()); - Assertions.assertNotEquals(0, actions.size()); - Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { - Assertions.assertTrue(goal.isBrokerAcceptable(broker)); - } - } - - @Test - public void testGoalActionAcceptanceScore() { - testActionAcceptanceScore(Resource.NW_IN); - testActionAcceptanceScore(Resource.NW_OUT); - } - - @Test - public void testSingleResourceCapacityOptimizeOneMove() { - testSingleResourceCapacityOptimizeOneMove(Resource.NW_IN); - testSingleResourceCapacityOptimizeOneMove(Resource.NW_OUT); - } - - @Test - public void testSingleResourceCapacityOptimizeMultiMove() { - testSingleResourceCapacityOptimizeMultiMove(Resource.NW_IN); - testSingleResourceCapacityOptimizeMultiMove(Resource.NW_OUT); - } - - @Test - public void testMultiGoalOptimizeWithOneToOneReplicaSwap() { - Goal goal = getGoalByResource(Resource.NW_IN); - Assertions.assertNotNull(goal); - - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - - broker0.setCapacity(Resource.NW_IN, 100); - broker0.setCapacity(Resource.NW_OUT, 100); - broker0.setLoad(Resource.NW_IN, 90); - broker0.setLoad(Resource.NW_OUT, 50); - - broker1.setCapacity(Resource.NW_IN, 100); - broker1.setCapacity(Resource.NW_OUT, 100); - broker1.setLoad(Resource.NW_IN, 20); - broker1.setLoad(Resource.NW_OUT, 90); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); - replica0.setLoad(Resource.NW_IN, 40); - replica0.setLoad(Resource.NW_OUT, 30); - replica1.setLoad(Resource.NW_IN, 50); - replica1.setLoad(Resource.NW_OUT, 20); - Assertions.assertEquals(90, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(50, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_1, 1); - replica2.setLoad(Resource.NW_IN, 5); - replica2.setLoad(Resource.NW_OUT, 50); - replica3.setLoad(Resource.NW_IN, 15); - replica3.setLoad(Resource.NW_OUT, 40); - Assertions.assertEquals(20, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); - Assertions.assertEquals(90, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - - List actions = goal.optimize(cluster, goalMap.values()); - Assertions.assertNotEquals(0, actions.size()); - Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { - Assertions.assertTrue(goal.isBrokerAcceptable(broker)); - } - } - - - @Test - public void testMultiGoalOptimizeWithOneToNReplicaSwap() { - //TODO: implement one-to-N replica swap - -// AbstractGoal goal = getGoalByResource(Resource.NW_IN); -// Assertions.assertNotNull(goal); -// -// ClusterModelSnapshot cluster = new ClusterModelSnapshot(); -// BrokerUpdater.Broker broker0 = createBroker(cluster, Resource.NW_IN, RACK, 0, true); -// BrokerUpdater.Broker broker1 = createBroker(cluster, Resource.NW_IN, RACK, 1, true); -// -// broker0.setCapacity(Resource.NW_IN, 100); -// broker0.setCapacity(Resource.NW_OUT, 100); -// broker0.setLoad(Resource.NW_IN, 90); -// broker0.setLoad(Resource.NW_OUT, 50); -// -// broker1.setCapacity(Resource.NW_IN, 100); -// broker1.setCapacity(Resource.NW_OUT, 100); -// broker1.setLoad(Resource.NW_IN, 20); -// broker1.setLoad(Resource.NW_OUT, 90); -// -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); -// replica0.setLoad(Resource.NW_IN, 40); -// replica0.setLoad(Resource.NW_OUT, 30); -// replica1.setLoad(Resource.NW_IN, 50); -// replica1.setLoad(Resource.NW_OUT, 20); -// Assertions.assertEquals(90, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); -// Assertions.assertEquals(20, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); -// -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 1); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_1, 1); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_2, 0); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_3, 0); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 1, TOPIC_4, 0); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_2, 1); -// TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_3, 1); -// replica2.setLoad(Resource.NW_IN, 1); -// replica2.setLoad(Resource.NW_OUT, 10); -// replica3.setLoad(Resource.NW_IN, 2); -// replica3.setLoad(Resource.NW_OUT, 15); -// replica4.setLoad(Resource.NW_IN, 3); -// replica4.setLoad(Resource.NW_OUT, 15); -// replica5.setLoad(Resource.NW_IN, 4); -// replica5.setLoad(Resource.NW_OUT, 15); -// replica6.setLoad(Resource.NW_IN, 5); -// replica6.setLoad(Resource.NW_OUT, 5); -// replica7.setLoad(Resource.NW_IN, 2); -// replica7.setLoad(Resource.NW_OUT, 15); -// replica8.setLoad(Resource.NW_IN, 3); -// replica8.setLoad(Resource.NW_OUT, 15); -// Assertions.assertEquals(20, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); -// Assertions.assertEquals(90, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); -// -// ClusterModelSnapshot optimizedCluster = goal.optimize(cluster, goalMap.values()); -// Assertions.assertNotNull(optimizedCluster); -// Assertions.assertEquals(0, optimizedCluster.replicasFor(4).size()); -// for (BrokerUpdater.Broker broker : optimizedCluster.brokers()) { -// Assertions.assertTrue(goal.isBrokerAcceptable(broker)); -// } - } -} diff --git a/core/src/test/java/kafka/autobalancer/goals/AbstractResourceDistributionGoalTest.java b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoalTest.java similarity index 54% rename from core/src/test/java/kafka/autobalancer/goals/AbstractResourceDistributionGoalTest.java rename to core/src/test/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoalTest.java index 36ff4672c9..bdcc2778dd 100644 --- a/core/src/test/java/kafka/autobalancer/goals/AbstractResourceDistributionGoalTest.java +++ b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceUsageDistributionGoalTest.java @@ -18,12 +18,11 @@ package kafka.autobalancer.goals; import kafka.autobalancer.common.Action; -import kafka.autobalancer.common.ActionType; import kafka.autobalancer.common.Resource; import kafka.autobalancer.config.AutoBalancerControllerConfig; -import kafka.autobalancer.model.BrokerUpdater; +import kafka.autobalancer.model.BrokerUpdater.Broker; import kafka.autobalancer.model.ClusterModelSnapshot; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; @@ -35,17 +34,17 @@ import java.util.StringJoiner; @Tag("S3Unit") -public class AbstractResourceDistributionGoalTest extends GoalTestBase { +public class AbstractResourceUsageDistributionGoalTest extends GoalTestBase { private final Map goalMap = new HashMap<>(); @BeforeEach public void setup() { Map config = new HashMap<>(); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, new StringJoiner(",") - .add(NetworkInDistributionGoal.class.getName()) - .add(NetworkOutDistributionGoal.class.getName()).toString()); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_THRESHOLD, 0.2); - config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_THRESHOLD, 0.2); + .add(NetworkInUsageDistributionGoal.class.getName()) + .add(NetworkOutUsageDistributionGoal.class.getName()).toString()); + config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_USAGE_DISTRIBUTION_DETECT_THRESHOLD, 0); + config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_USAGE_DISTRIBUTION_DETECT_THRESHOLD, 0); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_IN_DISTRIBUTION_DETECT_AVG_DEVIATION, 0.2); config.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_NETWORK_OUT_DISTRIBUTION_DETECT_AVG_DEVIATION, 0.2); AutoBalancerControllerConfig controllerConfig = new AutoBalancerControllerConfig(config, false); @@ -59,10 +58,10 @@ private Goal getGoalByResource(Resource resource) { Goal goal = null; switch (resource) { case NW_IN: - goal = goalMap.get(NetworkInDistributionGoal.class.getSimpleName()); + goal = goalMap.get(NetworkInUsageDistributionGoal.class.getSimpleName()); break; case NW_OUT: - goal = goalMap.get(NetworkOutDistributionGoal.class.getSimpleName()); + goal = goalMap.get(NetworkOutUsageDistributionGoal.class.getSimpleName()); break; default: break; @@ -70,67 +69,32 @@ private Goal getGoalByResource(Resource resource) { return goal; } - private void testActionAcceptanceScore(Resource resource) { - Goal goal = getGoalByResource(resource); - Assertions.assertNotNull(goal); - - ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); - BrokerUpdater.Broker broker2 = createBroker(cluster, RACK, 2, true); - broker1.setCapacity(resource, 100); - broker2.setCapacity(resource, 100); - - broker1.setLoad(resource, 20); - broker2.setLoad(resource, 80); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica = createTopicPartition(cluster, 1, TOPIC_0, 0); - replica.setLoad(resource, 20); - - Action action = new Action(ActionType.MOVE, replica.getTopicPartition(), broker1.getBrokerId(), broker2.getBrokerId()); - Assertions.assertEquals(0.4, goal.actionAcceptanceScore(action, cluster), 1e-15); - - broker1.setLoad(resource, 70); - broker2.setLoad(resource, 30); - Assertions.assertEquals(0.6, goal.actionAcceptanceScore(action, cluster), 1e-15); - - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 2, TOPIC_0, 1); - replica2.setLoad(resource, 10); - - Action action2 = new Action(ActionType.SWAP, replica.getTopicPartition(), broker1.getBrokerId(), broker2.getBrokerId(), replica2.getTopicPartition()); - Assertions.assertEquals(0.55, goal.actionAcceptanceScore(action2, cluster), 1e-15); - - replica.setLoad(resource, 5); - Assertions.assertEquals(0.475, goal.actionAcceptanceScore(action2, cluster), 1e-15); - } - private void testSingleResourceDistributionOptimizeOneMove(Resource resource) { Goal goal = getGoalByResource(resource); Assertions.assertNotNull(goal); ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); + Broker broker0 = createBroker(cluster, RACK, 0, true); + Broker broker1 = createBroker(cluster, RACK, 1, true); double load0 = 80; double load1 = 20; - broker0.setCapacity(resource, 100); broker0.setLoad(resource, load0); - broker1.setCapacity(resource, 80); broker1.setLoad(resource, load1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_2, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_3, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 1); + TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); + TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_2, 0); + TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_3, 0); + TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 1); replica0.setLoad(resource, 20); replica1.setLoad(resource, 30); replica2.setLoad(resource, 15); replica3.setLoad(resource, 15); Assertions.assertEquals(load0, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(resource)).sum()); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_4, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_2, 1); + TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_4, 0); + TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_2, 1); replica4.setLoad(resource, 15); replica5.setLoad(resource, 5); Assertions.assertEquals(load1, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(resource)).sum()); @@ -138,7 +102,7 @@ private void testSingleResourceDistributionOptimizeOneMove(Resource resource) { List actions = goal.optimize(cluster, goalMap.values()); Assertions.assertNotEquals(0, actions.size()); Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { + for (Broker broker : cluster.brokers()) { Assertions.assertTrue(goal.isBrokerAcceptable(broker)); } } @@ -148,24 +112,22 @@ private void testSingleResourceDistributionOptimizeMultiMoveOut(Resource resourc Assertions.assertNotNull(goal); ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); + Broker broker0 = createBroker(cluster, RACK, 0, true); + Broker broker1 = createBroker(cluster, RACK, 1, true); double load0 = 80; double load1 = 10; - broker0.setCapacity(resource, 100); broker0.setLoad(resource, load0); - broker1.setCapacity(resource, 100); broker1.setLoad(resource, load1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_0, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 3); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 0, TOPIC_0, 4); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 0, TOPIC_0, 5); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 0, TOPIC_0, 6); + TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); + TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_0, 1); + TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_0, 2); + TopicPartitionReplica replica3 = createTopicPartition(cluster, 0, TOPIC_0, 3); + TopicPartitionReplica replica4 = createTopicPartition(cluster, 0, TOPIC_0, 4); + TopicPartitionReplica replica5 = createTopicPartition(cluster, 0, TOPIC_0, 5); + TopicPartitionReplica replica6 = createTopicPartition(cluster, 0, TOPIC_0, 6); replica0.setLoad(resource, 10); replica1.setLoad(resource, 20); replica2.setLoad(resource, 15); @@ -175,8 +137,8 @@ private void testSingleResourceDistributionOptimizeMultiMoveOut(Resource resourc replica6.setLoad(resource, 15); Assertions.assertEquals(load0, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(resource)).sum()); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_1, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_1, 1); + TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_1, 0); + TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_1, 1); replica7.setLoad(resource, 5); replica8.setLoad(resource, 5); Assertions.assertEquals(load1, cluster.replicasFor(1).stream().mapToDouble(e -> e.load(resource)).sum()); @@ -184,7 +146,7 @@ private void testSingleResourceDistributionOptimizeMultiMoveOut(Resource resourc List actions = goal.optimize(cluster, goalMap.values()); Assertions.assertNotEquals(0, actions.size()); Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { + for (Broker broker : cluster.brokers()) { Assertions.assertTrue(goal.isBrokerAcceptable(broker)); } } @@ -194,30 +156,28 @@ private void testSingleResourceDistributionOptimizeMultiMoveIn(Resource resource Assertions.assertNotNull(goal); ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); + Broker broker0 = createBroker(cluster, RACK, 0, true); + Broker broker1 = createBroker(cluster, RACK, 1, true); double load0 = 10; double load1 = 80; - broker0.setCapacity(resource, 100); broker0.setLoad(resource, load0); - broker1.setCapacity(resource, 100); broker1.setLoad(resource, load1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_1, 1); + TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); + TopicPartitionReplica replica2 = createTopicPartition(cluster, 0, TOPIC_1, 1); replica1.setLoad(resource, 5); replica2.setLoad(resource, 5); Assertions.assertEquals(load0, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(resource)).sum()); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_0, 2); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica6 = createTopicPartition(cluster, 1, TOPIC_0, 3); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_0, 4); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_0, 5); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica9 = createTopicPartition(cluster, 1, TOPIC_0, 6); + TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_0, 0); + TopicPartitionReplica replica4 = createTopicPartition(cluster, 1, TOPIC_0, 1); + TopicPartitionReplica replica5 = createTopicPartition(cluster, 1, TOPIC_0, 2); + TopicPartitionReplica replica6 = createTopicPartition(cluster, 1, TOPIC_0, 3); + TopicPartitionReplica replica7 = createTopicPartition(cluster, 1, TOPIC_0, 4); + TopicPartitionReplica replica8 = createTopicPartition(cluster, 1, TOPIC_0, 5); + TopicPartitionReplica replica9 = createTopicPartition(cluster, 1, TOPIC_0, 6); replica3.setLoad(resource, 10); replica4.setLoad(resource, 20); replica5.setLoad(resource, 15); @@ -230,17 +190,11 @@ private void testSingleResourceDistributionOptimizeMultiMoveIn(Resource resource List actions = goal.optimize(cluster, goalMap.values()); Assertions.assertNotEquals(0, actions.size()); Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { + for (Broker broker : cluster.brokers()) { Assertions.assertTrue(goal.isBrokerAcceptable(broker)); } } - @Test - public void testGoalActionAcceptanceScore() { - testActionAcceptanceScore(Resource.NW_IN); - testActionAcceptanceScore(Resource.NW_OUT); - } - @Test public void testSingleResourceDistributionOptimizeOneMove() { testSingleResourceDistributionOptimizeOneMove(Resource.NW_IN); @@ -260,21 +214,17 @@ private void testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource resource) { Assertions.assertNotNull(goal); ClusterModelSnapshot cluster = new ClusterModelSnapshot(); - BrokerUpdater.Broker broker0 = createBroker(cluster, RACK, 0, true); - BrokerUpdater.Broker broker1 = createBroker(cluster, RACK, 1, true); + Broker broker0 = createBroker(cluster, RACK, 0, true); + Broker broker1 = createBroker(cluster, RACK, 1, true); - broker0.setCapacity(Resource.NW_IN, 100); - broker0.setCapacity(Resource.NW_OUT, 100); broker0.setLoad(Resource.NW_IN, 90); broker0.setLoad(Resource.NW_OUT, 50); - broker1.setCapacity(Resource.NW_IN, 100); - broker1.setCapacity(Resource.NW_OUT, 100); broker1.setLoad(Resource.NW_IN, 20); broker1.setLoad(Resource.NW_OUT, 90); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); + TopicPartitionReplica replica0 = createTopicPartition(cluster, 0, TOPIC_0, 0); + TopicPartitionReplica replica1 = createTopicPartition(cluster, 0, TOPIC_1, 0); replica0.setLoad(Resource.NW_IN, 40); replica0.setLoad(Resource.NW_OUT, 30); replica1.setLoad(Resource.NW_IN, 50); @@ -282,8 +232,8 @@ private void testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource resource) { Assertions.assertEquals(90, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_IN)).sum()); Assertions.assertEquals(50, cluster.replicasFor(0).stream().mapToDouble(e -> e.load(Resource.NW_OUT)).sum()); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 1); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_1, 1); + TopicPartitionReplica replica2 = createTopicPartition(cluster, 1, TOPIC_0, 1); + TopicPartitionReplica replica3 = createTopicPartition(cluster, 1, TOPIC_1, 1); replica2.setLoad(Resource.NW_IN, 5); replica2.setLoad(Resource.NW_OUT, 50); replica3.setLoad(Resource.NW_IN, 15); @@ -294,7 +244,7 @@ private void testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource resource) { List actions = goal.optimize(cluster, goalMap.values()); Assertions.assertNotEquals(0, actions.size()); Assertions.assertNotNull(cluster); - for (BrokerUpdater.Broker broker : cluster.brokers()) { + for (Broker broker : cluster.brokers()) { Assertions.assertTrue(goal.isBrokerAcceptable(broker)); } } diff --git a/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java b/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java index 134c7a20cf..f6f422de6e 100644 --- a/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java +++ b/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java @@ -17,9 +17,9 @@ package kafka.autobalancer.goals; -import kafka.autobalancer.model.BrokerUpdater; +import kafka.autobalancer.model.BrokerUpdater.Broker; import kafka.autobalancer.model.ClusterModelSnapshot; -import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Tag; @@ -32,21 +32,20 @@ public class GoalTestBase { protected static final String TOPIC_3 = "TestTopic3"; protected static final String TOPIC_4 = "TestTopic4"; - protected BrokerUpdater.Broker createBroker(ClusterModelSnapshot cluster, String rack, - int brokerId, boolean active) { - BrokerUpdater.Broker broker = new BrokerUpdater.Broker(brokerId); - broker.setActive(active); - cluster.addBroker(rack, broker); + protected Broker createBroker(ClusterModelSnapshot cluster, String rack, + int brokerId, boolean active) { + Broker broker = new Broker(brokerId, active); + cluster.addBroker(brokerId, rack, broker); return broker; } - protected TopicPartitionReplicaUpdater.TopicPartitionReplica createTopicPartition(ClusterModelSnapshot cluster, + protected TopicPartitionReplica createTopicPartition(ClusterModelSnapshot cluster, int brokerId, String topic, int partition) { TopicPartition tp = new TopicPartition(topic, partition); - TopicPartitionReplicaUpdater.TopicPartitionReplica replica = new TopicPartitionReplicaUpdater.TopicPartitionReplica(tp); - cluster.addTopicPartition(brokerId, replica); + TopicPartitionReplica replica = new TopicPartitionReplica(tp); + cluster.addTopicPartition(brokerId, tp, replica); return replica; } } diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java index 05fd1115dd..4925f9d13d 100644 --- a/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java @@ -37,7 +37,6 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import java.lang.reflect.Field; import java.time.Duration; import java.util.Arrays; import java.util.Collections; @@ -47,19 +46,10 @@ import java.util.Properties; import java.util.Set; -import static kafka.autobalancer.common.RawMetricType.ALL_TOPIC_BYTES_IN; -import static kafka.autobalancer.common.RawMetricType.ALL_TOPIC_BYTES_OUT; -import static kafka.autobalancer.common.RawMetricType.BROKER_CAPACITY_NW_IN; -import static kafka.autobalancer.common.RawMetricType.BROKER_CAPACITY_NW_OUT; -import static kafka.autobalancer.common.RawMetricType.BROKER_CPU_UTIL; import static kafka.autobalancer.common.RawMetricType.PARTITION_SIZE; import static kafka.autobalancer.common.RawMetricType.TOPIC_PARTITION_BYTES_IN; import static kafka.autobalancer.common.RawMetricType.TOPIC_PARTITION_BYTES_OUT; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; - @Tag("S3Unit") public class AutoBalancerMetricsReporterTest extends AutoBalancerClientsIntegrationTestHarness { @@ -107,99 +97,28 @@ public void testReportingMetrics() { try (Consumer consumer = new KafkaConsumer<>(props)) { consumer.subscribe(Collections.singleton(METRIC_TOPIC)); long startMs = System.currentTimeMillis(); - Set expectedBrokerMetricTypes = new HashSet<>(Arrays.asList( - (int) BROKER_CAPACITY_NW_IN.id(), - (int) BROKER_CAPACITY_NW_OUT.id(), - (int) ALL_TOPIC_BYTES_IN.id(), - (int) ALL_TOPIC_BYTES_OUT.id(), - (int) BROKER_CPU_UTIL.id())); Set expectedTopicPartitionMetricTypes = new HashSet<>(Arrays.asList( (int) TOPIC_PARTITION_BYTES_IN.id(), (int) TOPIC_PARTITION_BYTES_OUT.id(), (int) PARTITION_SIZE.id())); - Set expectedMetricTypes = new HashSet<>(expectedBrokerMetricTypes); - expectedMetricTypes.addAll(expectedTopicPartitionMetricTypes); + Set expectedMetricTypes = new HashSet<>(expectedTopicPartitionMetricTypes); Set metricTypes = new HashSet<>(); ConsumerRecords records; - while (metricTypes.size() < (expectedBrokerMetricTypes.size() + expectedTopicPartitionMetricTypes.size()) - && System.currentTimeMillis() < startMs + 15000) { + while (metricTypes.size() < expectedTopicPartitionMetricTypes.size() && System.currentTimeMillis() < startMs + 15000) { records = consumer.poll(Duration.ofMillis(10L)); for (ConsumerRecord record : records) { - AutoBalancerMetrics metrics = record.value(); Set localMetricTypes = new HashSet<>(); - for (RawMetricType type : record.value().getMetricTypeValueMap().keySet()) { + for (RawMetricType type : record.value().getMetricValueMap().keySet()) { int typeId = type.id(); metricTypes.add(typeId); localMetricTypes.add(typeId); } - Set expectedMap = metrics.metricClassId() == AutoBalancerMetrics.MetricClassId.BROKER_METRIC ? - expectedBrokerMetricTypes : expectedTopicPartitionMetricTypes; - Assertions.assertEquals(expectedMap, localMetricTypes, "Expected " + expectedMap + ", but saw " + localMetricTypes); + Assertions.assertEquals(expectedTopicPartitionMetricTypes, localMetricTypes, + "Expected " + expectedTopicPartitionMetricTypes + ", but saw " + localMetricTypes); } } Assertions.assertEquals(expectedMetricTypes, metricTypes, "Expected " + expectedMetricTypes + ", but saw " + metricTypes); } } - - @Test - public void testConfigureNwCapacity() throws NoSuchFieldException, IllegalAccessException { - // Create a spy object for AutoBalancerMetricsReporter - AutoBalancerMetricsReporter autoBalancerMetricsReporter = spy(new AutoBalancerMetricsReporter()); - - // When calling the autoBalancerMetricsReporter.createAutoBalancerMetricsProducer method, no action is taken - doNothing().when(autoBalancerMetricsReporter).createAutoBalancerMetricsProducer(mock(Properties.class)); - - // Using java reflection to obtain private fields - Field brokerNwInCapacity = AutoBalancerMetricsReporter.class.getDeclaredField("brokerNwInCapacity"); - Field brokerNwOutCapacity = AutoBalancerMetricsReporter.class.getDeclaredField("brokerNwOutCapacity"); - brokerNwInCapacity.setAccessible(true); - brokerNwOutCapacity.setAccessible(true); - - // init default configs - Map initConfigs = Map.of( - "broker.id", "1", - "broker.rack", "rack-a" - ); - Map configs = new HashMap<>(initConfigs); - - //test1.the priority of AUTO_BALANCER_BROKER_NW_IN/OUT_CAPACITY is highest - configs.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_IN_CAPACITY, 51200); - configs.put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, 51200); - configs.put(KafkaConfig.S3NetworkBaselineBandwidthProp(), 102400); - - autoBalancerMetricsReporter.configure(configs); - - Assertions.assertEquals(51200, brokerNwInCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + 51200 + ", but saw " + brokerNwInCapacity.getDouble(autoBalancerMetricsReporter)); - Assertions.assertEquals(51200, brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + 51200 + ", but saw " + brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter)); - - - //test2.the priority of S3NetworkBaselineBandwidthProp is second highest - configs = new HashMap<>(initConfigs); - configs.put(KafkaConfig.S3NetworkBaselineBandwidthProp(), 204800); - - autoBalancerMetricsReporter.configure(configs); - - Assertions.assertEquals(204800, brokerNwInCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + 204800 + ", but saw " + brokerNwInCapacity.getDouble(autoBalancerMetricsReporter)); - Assertions.assertEquals(204800, brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + 204800 + ", but saw " + brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter)); - - - //test3.default autobalancer config is last one - configs = new HashMap<>(initConfigs); - autoBalancerMetricsReporter.configure(configs); - - Assertions.assertEquals(AutoBalancerMetricsReporterConfig.DEFAULT_AUTO_BALANCER_BROKER_NW_IN_CAPACITY, - brokerNwInCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + AutoBalancerMetricsReporterConfig.DEFAULT_AUTO_BALANCER_BROKER_NW_IN_CAPACITY - + ", but saw " + brokerNwInCapacity.getDouble(autoBalancerMetricsReporter)); - Assertions.assertEquals(AutoBalancerMetricsReporterConfig.DEFAULT_AUTO_BALANCER_BROKER_NW_OUT_CAPACITY, - brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter), - "Expected " + AutoBalancerMetricsReporterConfig.DEFAULT_AUTO_BALANCER_BROKER_NW_OUT_CAPACITY - + ", but saw " + brokerNwOutCapacity.getDouble(autoBalancerMetricsReporter)); - - } } diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java index a51fbf206c..e347299afc 100644 --- a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java @@ -37,26 +37,6 @@ public class MetricSerdeTest { private static final double VALUE = 0.1; private static final double VALUE1 = 0.2; - @Test - public void testBrokerMetricSerde() throws UnknownVersionException { - AutoBalancerMetrics brokerMetric = new BrokerMetrics(123L, 0, "") - .put(RawMetricType.ALL_TOPIC_BYTES_IN, VALUE) - .put(RawMetricType.ALL_TOPIC_BYTES_OUT, VALUE1); - AutoBalancerMetrics deserialized = MetricSerde.fromBytes(MetricSerde.toBytes(brokerMetric)); - assertNotNull(deserialized); - assertEquals(AutoBalancerMetrics.MetricClassId.BROKER_METRIC.id(), deserialized.metricClassId().id()); - Map metricMap = deserialized.getMetricTypeValueMap(); - assertEquals(2, metricMap.size()); - assertTrue(metricMap.containsKey(RawMetricType.ALL_TOPIC_BYTES_IN)); - assertTrue(metricMap.containsKey(RawMetricType.ALL_TOPIC_BYTES_OUT)); - assertEquals(VALUE, metricMap.get(RawMetricType.ALL_TOPIC_BYTES_IN), 0.000001); - assertEquals(VALUE1, metricMap.get(RawMetricType.ALL_TOPIC_BYTES_OUT), 0.000001); - assertEquals(TIME, deserialized.time()); - assertEquals(BROKER_ID, deserialized.brokerId()); - assertEquals("", deserialized.brokerRack()); - - } - @Test public void testPartitionMetricSerde() throws UnknownVersionException { AutoBalancerMetrics topicPartitionMetrics = new TopicPartitionMetrics(123L, 0, "", TOPIC, PARTITION) @@ -64,8 +44,8 @@ public void testPartitionMetricSerde() throws UnknownVersionException { .put(RawMetricType.TOPIC_PARTITION_BYTES_IN, VALUE1); AutoBalancerMetrics deserialized = MetricSerde.fromBytes(MetricSerde.toBytes(topicPartitionMetrics)); assertNotNull(deserialized); - assertEquals(AutoBalancerMetrics.MetricClassId.PARTITION_METRIC.id(), deserialized.metricClassId().id()); - Map metricMap = deserialized.getMetricTypeValueMap(); + assertEquals(MetricClassId.PARTITION_METRIC.id(), deserialized.metricClassId().id()); + Map metricMap = deserialized.getMetricValueMap(); assertEquals(2, metricMap.size()); assertTrue(metricMap.containsKey(RawMetricType.PARTITION_SIZE)); assertTrue(metricMap.containsKey(RawMetricType.TOPIC_PARTITION_BYTES_IN)); diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java index 7cc556767f..5cee88b2e0 100644 --- a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java @@ -25,21 +25,6 @@ @Tag("S3Unit") public class MetricsUtilsTest { - @Test - public void testSanityCheckBrokerMetricsCompleteness() { - BrokerMetrics metrics = new BrokerMetrics(System.currentTimeMillis(), 1, ""); - metrics.put(RawMetricType.BROKER_CAPACITY_NW_IN, 10); - Assertions.assertFalse(MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)); - metrics.put(RawMetricType.BROKER_CAPACITY_NW_OUT, 10); - Assertions.assertFalse(MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)); - metrics.put(RawMetricType.ALL_TOPIC_BYTES_IN, 10); - Assertions.assertFalse(MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)); - metrics.put(RawMetricType.ALL_TOPIC_BYTES_OUT, 10); - Assertions.assertFalse(MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)); - metrics.put(RawMetricType.BROKER_CPU_UTIL, 10); - Assertions.assertTrue(MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)); - } - @Test public void testSanityCheckTopicPartitionMetricsCompleteness() { TopicPartitionMetrics metrics = new TopicPartitionMetrics(System.currentTimeMillis(), 1, "", "testTopic", 0); diff --git a/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java b/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java index 762d2d4030..495fbcaa95 100644 --- a/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java +++ b/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java @@ -18,7 +18,6 @@ package kafka.autobalancer.model; import kafka.autobalancer.common.RawMetricType; -import kafka.autobalancer.metricsreporter.metric.BrokerMetrics; import kafka.autobalancer.metricsreporter.metric.TopicPartitionMetrics; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; @@ -48,8 +47,8 @@ public void testRegisterBroker() { clusterModel.onBrokerRegister(record2); clusterModel.onBrokerRegister(record1); - Assertions.assertEquals(1, clusterModel.brokerUpdater(1).get().getBrokerId()); - Assertions.assertEquals(2, clusterModel.brokerUpdater(2).get().getBrokerId()); + Assertions.assertEquals(1, ((BrokerUpdater.Broker) clusterModel.brokerUpdater(1).get()).getBrokerId()); + Assertions.assertEquals(2, ((BrokerUpdater.Broker) clusterModel.brokerUpdater(2).get()).getBrokerId()); } @Test @@ -63,14 +62,14 @@ public void testUnregisterBroker() { clusterModel.onBrokerRegister(record2); clusterModel.onBrokerRegister(record1); - Assertions.assertEquals(1, clusterModel.brokerUpdater(1).get().getBrokerId()); - Assertions.assertEquals(2, clusterModel.brokerUpdater(2).get().getBrokerId()); + Assertions.assertEquals(1, ((BrokerUpdater.Broker) clusterModel.brokerUpdater(1).get()).getBrokerId()); + Assertions.assertEquals(2, ((BrokerUpdater.Broker) clusterModel.brokerUpdater(2).get()).getBrokerId()); UnregisterBrokerRecord unregisterRecord = new UnregisterBrokerRecord() .setBrokerId(2); clusterModel.onBrokerUnregister(unregisterRecord); - Assertions.assertEquals(1, clusterModel.brokerUpdater(1).get().getBrokerId()); + Assertions.assertEquals(1, ((BrokerUpdater.Broker) clusterModel.brokerUpdater(1).get()).getBrokerId()); Assertions.assertNull(clusterModel.brokerUpdater(2)); } @@ -192,28 +191,6 @@ public void testChangePartition() { Assertions.assertNull(clusterModel.replicaUpdater(oldBrokerId, tp)); } - @Test - public void testUpdateBroker() { - RecordClusterModel clusterModel = new RecordClusterModel(); - int brokerId = 1; - - // update on non-exist broker - long now = System.currentTimeMillis(); - BrokerMetrics brokerMetrics = new BrokerMetrics(now, brokerId, ""); - brokerMetrics.put(RawMetricType.BROKER_CAPACITY_NW_IN, 10); - brokerMetrics.put(RawMetricType.BROKER_CAPACITY_NW_OUT, 10); - brokerMetrics.put(RawMetricType.ALL_TOPIC_BYTES_IN, 10); - brokerMetrics.put(RawMetricType.ALL_TOPIC_BYTES_OUT, 10); - brokerMetrics.put(RawMetricType.BROKER_CPU_UTIL, 10); - Assertions.assertFalse(clusterModel.updateBrokerMetrics(brokerMetrics.brokerId(), brokerMetrics.getMetricTypeValueMap(), brokerMetrics.time())); - - RegisterBrokerRecord registerBrokerRecord = new RegisterBrokerRecord() - .setBrokerId(brokerId); - clusterModel.onBrokerRegister(registerBrokerRecord); - Assertions.assertEquals(brokerId, clusterModel.brokerUpdater(brokerId).get().getBrokerId()); - Assertions.assertTrue(clusterModel.updateBrokerMetrics(brokerMetrics.brokerId(), brokerMetrics.getMetricTypeValueMap(), brokerMetrics.time())); - } - @Test public void testUpdatePartition() { RecordClusterModel clusterModel = new RecordClusterModel(); @@ -229,7 +206,7 @@ public void testUpdatePartition() { topicPartitionMetrics.put(RawMetricType.TOPIC_PARTITION_BYTES_OUT, 10); topicPartitionMetrics.put(RawMetricType.PARTITION_SIZE, 10); Assertions.assertFalse(clusterModel.updateTopicPartitionMetrics(topicPartitionMetrics.brokerId(), - new TopicPartition(topicName, partition), topicPartitionMetrics.getMetricTypeValueMap(), topicPartitionMetrics.time())); + new TopicPartition(topicName, partition), topicPartitionMetrics.getMetricValueMap(), topicPartitionMetrics.time())); RegisterBrokerRecord registerBrokerRecord = new RegisterBrokerRecord() .setBrokerId(brokerId); @@ -244,7 +221,7 @@ public void testUpdatePartition() { .setPartitionId(partition); clusterModel.onPartitionCreate(partitionRecord); Assertions.assertTrue(clusterModel.updateTopicPartitionMetrics(topicPartitionMetrics.brokerId(), - new TopicPartition(topicName, partition), topicPartitionMetrics.getMetricTypeValueMap(), topicPartitionMetrics.time())); + new TopicPartition(topicName, partition), topicPartitionMetrics.getMetricValueMap(), topicPartitionMetrics.time())); } @Test diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties index 3eb176ae75..e213b4f09a 100644 --- a/tests/kafkatest/services/kafka/templates/kafka.properties +++ b/tests/kafkatest/services/kafka/templates/kafka.properties @@ -24,9 +24,9 @@ process.roles=controller autobalancer.controller.enable=true autobalancer.controller.anomaly.detect.interval.ms=60000 autobalancer.controller.metrics.delay.ms=20000 -autobalancer.controller.network.in.distribution.detect.threshold=0.2 +autobalancer.controller.network.in.usage.distribution.detect.threshold=0.2 autobalancer.controller.network.in.distribution.detect.avg.deviation=0.2 -autobalancer.controller.network.out.distribution.detect.threshold=0.2 +autobalancer.controller.network.out.usage.distribution.detect.threshold=0.2 autobalancer.controller.network.out.distribution.detect.avg.deviation=0.2 autobalancer.controller.network.in.utilization.threshold=0.8 autobalancer.controller.network.out.utilization.threshold=0.8