diff --git a/core/src/main/java/kafka/autobalancer/AnomalyDetector.java b/core/src/main/java/kafka/autobalancer/AnomalyDetector.java new file mode 100644 index 0000000000..1faaa7e044 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/AnomalyDetector.java @@ -0,0 +1,128 @@ +/* + * 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; + +import kafka.autobalancer.common.Action; +import kafka.autobalancer.common.AutoBalancerThreadFactory; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.goals.AbstractGoal; +import kafka.autobalancer.model.BrokerUpdater; +import kafka.autobalancer.model.ClusterModel; +import kafka.autobalancer.model.ClusterModelSnapshot; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.controller.QuorumController; +import org.slf4j.Logger; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class AnomalyDetector { + private final Logger logger; + private final List goalsByPriority; + private final ClusterModel clusterModel; + private final ScheduledExecutorService executorService; + private final ExecutionManager executionManager; + private final long detectInterval; + private final Set excludedBrokers = new HashSet<>(); + private final Set excludedTopics = new HashSet<>(); + private volatile boolean running; + + public AnomalyDetector(AutoBalancerControllerConfig config, QuorumController quorumController, ClusterModel clusterModel, LogContext logContext) { + if (logContext == null) { + logContext = new LogContext("[AnomalyDetector] "); + } + this.logger = logContext.logger(AnomalyDetector.class); + this.goalsByPriority = config.getConfiguredInstances(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, AbstractGoal.class); + Collections.sort(this.goalsByPriority); + logger.info("Goals: {}", this.goalsByPriority); + this.detectInterval = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS); + fetchExcludedConfig(config); + this.clusterModel = clusterModel; + this.executorService = Executors.newSingleThreadScheduledExecutor(new AutoBalancerThreadFactory("anomaly-detector")); + this.executionManager = new ExecutionManager(config, quorumController, + new LogContext(String.format("[ExecutionManager id=%d] ", quorumController.nodeId()))); + this.running = false; + } + + private void fetchExcludedConfig(AutoBalancerControllerConfig config) { + List brokerIds = config.getList(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS); + for (String brokerIdStr : brokerIds) { + try { + excludedBrokers.add(Integer.parseInt(brokerIdStr)); + } catch (NumberFormatException ignored) { + + } + } + List topics = config.getList(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS); + excludedTopics.addAll(topics); + logger.info("Excluded brokers: {}, excluded topics: {}", excludedBrokers, excludedTopics); + } + + public void start() { + this.executionManager.start(); + this.executorService.scheduleWithFixedDelay(this::detect, detectInterval, detectInterval, TimeUnit.MILLISECONDS); + logger.info("Started"); + } + + public void shutdown() throws InterruptedException { + this.running = false; + this.executorService.shutdown(); + this.executionManager.shutdown(); + logger.info("Shutdown completed"); + } + + public void pause() { + this.running = false; + } + + public void resume() { + this.running = true; + } + + private 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); + + for (BrokerUpdater.Broker broker : snapshot.brokers()) { + logger.info("Broker status: {}", broker); + for (TopicPartitionReplicaUpdater.TopicPartitionReplica replica : snapshot.replicasFor(broker.getBrokerId())) { + logger.debug("Replica status {}", replica); + } + } + + for (AbstractGoal goal : goalsByPriority) { + if (!this.running) { + break; + } + List actions = goal.optimize(snapshot, goalsByPriority); + logger.debug("Optimized actions {} for goal {}", actions, goal.name()); + this.executionManager.appendActions(actions); + } + // TODO: wait for completion of all actions before next detect + } +} diff --git a/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java b/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java new file mode 100644 index 0000000000..95d13584ee --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/AutoBalancerManager.java @@ -0,0 +1,194 @@ +/* + * 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; + +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.listeners.BrokerStatusListener; +import kafka.autobalancer.listeners.ClusterStatusListenerRegistry; +import kafka.autobalancer.listeners.TopicPartitionStatusListener; +import kafka.autobalancer.model.ClusterModel; +import kafka.server.KafkaConfig; +import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.MetadataRecordType; +import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RemoveTopicRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.controller.QuorumController; +import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.KafkaRaftClient; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.snapshot.SnapshotReader; +import org.slf4j.Logger; + +import java.util.List; + +public class AutoBalancerManager { + private final Logger logger; + private final LoadRetriever loadRetriever; + private final AnomalyDetector anomalyDetector; + private final KafkaEventQueue queue; + private final QuorumController quorumController; + + public AutoBalancerManager(Time time, KafkaConfig kafkaConfig, QuorumController quorumController, KafkaRaftClient raftClient) { + LogContext logContext = new LogContext(String.format("[AutoBalancerManager id=%d] ", quorumController.nodeId())); + logger = logContext.logger(AutoBalancerManager.class); + AutoBalancerControllerConfig config = new AutoBalancerControllerConfig(kafkaConfig.props(), false); + ClusterModel clusterModel = new ClusterModel(config, new LogContext(String.format("[ClusterModel id=%d] ", quorumController.nodeId()))); + this.loadRetriever = new LoadRetriever(config, quorumController, clusterModel, + new LogContext(String.format("[LoadRetriever id=%d] ", quorumController.nodeId()))); + this.anomalyDetector = new AnomalyDetector(config, quorumController, clusterModel, + new LogContext(String.format("[AnomalyDetector id=%d] ", quorumController.nodeId()))); + this.queue = new KafkaEventQueue(time, new LogContext(), "auto-balancer-"); + this.quorumController = quorumController; + ClusterStatusListenerRegistry registry = new ClusterStatusListenerRegistry(); + registry.register((BrokerStatusListener) clusterModel); + registry.register((TopicPartitionStatusListener) clusterModel); + registry.register(this.loadRetriever); + raftClient.register(new AutoBalancerListener(registry, this.loadRetriever, this.anomalyDetector)); + } + + public void start() { + loadRetriever.start(); + anomalyDetector.start(); + logger.info("Started"); + } + + public void shutdown() throws InterruptedException { + anomalyDetector.shutdown(); + loadRetriever.shutdown(); + queue.close(); + logger.info("Shutdown completed"); + } + + class AutoBalancerListener implements RaftClient.Listener { + private final ClusterStatusListenerRegistry registry; + private final LoadRetriever loadRetriever; + private final AnomalyDetector anomalyDetector; + + public AutoBalancerListener(ClusterStatusListenerRegistry registry, LoadRetriever loadRetriever, AnomalyDetector anomalyDetector) { + this.registry = registry; + this.loadRetriever = loadRetriever; + this.anomalyDetector = anomalyDetector; + } + + private void handleMessage(ApiMessage message) { + MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); + switch (type) { + case REGISTER_BROKER_RECORD: + for (BrokerStatusListener listener : this.registry.brokerListeners()) { + listener.onBrokerRegister((RegisterBrokerRecord) message); + } + break; + case UNREGISTER_BROKER_RECORD: + for (BrokerStatusListener listener : this.registry.brokerListeners()) { + listener.onBrokerUnregister((UnregisterBrokerRecord) message); + } + break; + case BROKER_REGISTRATION_CHANGE_RECORD: + for (BrokerStatusListener listener : this.registry.brokerListeners()) { + listener.onBrokerRegistrationChanged((BrokerRegistrationChangeRecord) message); + } + break; + case TOPIC_RECORD: + for (TopicPartitionStatusListener listener : this.registry.topicPartitionListeners()) { + listener.onTopicCreate((TopicRecord) message); + } + break; + case REMOVE_TOPIC_RECORD: + for (TopicPartitionStatusListener listener : this.registry.topicPartitionListeners()) { + listener.onTopicDelete((RemoveTopicRecord) message); + } + break; + case PARTITION_RECORD: + for (TopicPartitionStatusListener listener : this.registry.topicPartitionListeners()) { + listener.onPartitionCreate((PartitionRecord) message); + } + break; + case PARTITION_CHANGE_RECORD: + for (TopicPartitionStatusListener listener : this.registry.topicPartitionListeners()) { + listener.onPartitionChange((PartitionChangeRecord) message); + } + break; + default: + break; + } + } + + @Override + public void handleCommit(BatchReader reader) { + queue.append(() -> { + try (reader) { + while (reader.hasNext()) { + Batch batch = reader.next(); + List messages = batch.records(); + for (ApiMessageAndVersion apiMessage : messages) { + handleMessage(apiMessage.message()); + } + } + } + }); + } + + @Override + public void handleSnapshot(SnapshotReader reader) { + queue.append(() -> { + try (reader) { + while (reader.hasNext()) { + Batch batch = reader.next(); + List messages = batch.records(); + for (ApiMessageAndVersion apiMessage : messages) { + handleMessage(apiMessage.message()); + } + } + } + }); + } + + @Override + public void handleLeaderChange(LeaderAndEpoch leader) { + queue.append(() -> { + if (leader.leaderId().isEmpty()) { + return; + } + boolean isLeader = leader.isLeader(quorumController.nodeId()); + if (isLeader) { + this.anomalyDetector.resume(); + } else { + this.anomalyDetector.pause(); + } + this.loadRetriever.onLeaderChanged(isLeader); + }); + } + + @Override + public void beginShutdown() { + RaftClient.Listener.super.beginShutdown(); + } + } + +} diff --git a/core/src/main/java/kafka/autobalancer/ExecutionManager.java b/core/src/main/java/kafka/autobalancer/ExecutionManager.java new file mode 100644 index 0000000000..3b35b9202c --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/ExecutionManager.java @@ -0,0 +1,130 @@ +/* + * 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; + +import kafka.autobalancer.common.Action; +import kafka.autobalancer.common.ActionType; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.ControllerRequestContext; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.OptionalLong; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +public class ExecutionManager implements Runnable { + private final Logger logger; + private final Controller controller; + private final BlockingQueue actionQueue = new ArrayBlockingQueue<>(1000); + private final long executionInterval; + private final KafkaThread dispatchThread; + // TODO: optimize to per-broker concurrency control + private long lastExecutionTime = 0L; + private volatile boolean shutdown; + + public ExecutionManager(AutoBalancerControllerConfig config, Controller controller) { + this(config, controller, null); + } + + public ExecutionManager(AutoBalancerControllerConfig config, Controller controller, LogContext logContext) { + if (logContext == null) { + logContext = new LogContext("[ExecutionManager] "); + } + this.logger = logContext.logger(ExecutionManager.class); + this.controller = controller; + this.executionInterval = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS); + this.dispatchThread = KafkaThread.daemon("executor-dispatcher", this); + } + + public void start() { + this.shutdown = false; + this.dispatchThread.start(); + logger.info("Started"); + } + + public void shutdown() { + this.shutdown = true; + this.dispatchThread.interrupt(); + logger.info("Shutdown completed"); + } + + @Override + public void run() { + while (!shutdown) { + try { + Action action = actionQueue.take(); + long now = System.currentTimeMillis(); + long nextExecutionTime = lastExecutionTime + executionInterval; + while (!shutdown && lastExecutionTime != 0 && now < nextExecutionTime) { + try { + Thread.sleep(nextExecutionTime - now); + } catch (InterruptedException ignored) { + break; + } + now = System.currentTimeMillis(); + } + if (shutdown) { + break; + } + ControllerRequestContext context = new ControllerRequestContext(null, null, OptionalLong.empty()); + AlterPartitionReassignmentsRequestData request = new AlterPartitionReassignmentsRequestData(); + List topicList = new ArrayList<>(); + topicList.add(buildTopic(action.getSrcTopicPartition(), action.getDestBrokerId())); + if (action.getType() == ActionType.SWAP) { + topicList.add(buildTopic(action.getDestTopicPartition(), action.getSrcBrokerId())); + } + request.setTopics(topicList); + this.controller.alterPartitionReassignments(context, request); + lastExecutionTime = Time.SYSTEM.milliseconds(); + logger.info("Executing {}", action.prettyString()); + } catch (InterruptedException ignored) { + + } + } + } + + private AlterPartitionReassignmentsRequestData.ReassignableTopic buildTopic(TopicPartition tp, int brokerId) { + String topicName = tp.topic(); + AlterPartitionReassignmentsRequestData.ReassignableTopic topic = new AlterPartitionReassignmentsRequestData.ReassignableTopic() + .setName(topicName) + .setPartitions(new ArrayList<>()); + AlterPartitionReassignmentsRequestData.ReassignablePartition partition = new AlterPartitionReassignmentsRequestData.ReassignablePartition(); + partition.setPartitionIndex(tp.partition()); + partition.setReplicas(List.of(brokerId)); + topic.setPartitions(List.of(partition)); + return topic; + } + + public void appendActions(List actions) { + for (Action action : actions) { + try { + this.actionQueue.put(action); + } catch (InterruptedException ignored) { + + } + } + } +} diff --git a/core/src/main/java/kafka/autobalancer/LoadRetriever.java b/core/src/main/java/kafka/autobalancer/LoadRetriever.java new file mode 100644 index 0000000000..8d85add1e2 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/LoadRetriever.java @@ -0,0 +1,385 @@ +/* + * 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; + +import kafka.autobalancer.common.AutoBalancerThreadFactory; +import kafka.autobalancer.config.AutoBalancerConfig; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.listeners.BrokerStatusListener; +import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics; +import kafka.autobalancer.metricsreporter.metric.BrokerMetrics; +import kafka.autobalancer.metricsreporter.metric.MetricSerde; +import kafka.autobalancer.metricsreporter.metric.TopicPartitionMetrics; +import kafka.autobalancer.model.ClusterModel; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.message.CreateTopicsRequestData; +import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.ControllerRequestContext; +import org.apache.kafka.metadata.BrokerRegistrationFencingChange; +import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange; +import org.slf4j.Logger; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.OptionalLong; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class LoadRetriever implements Runnable, BrokerStatusListener { + public static final Random RANDOM = new Random(); + private final Logger logger; + private final Map bootstrapServerMap; + private final String metricReporterTopic; + private final int metricReporterTopicPartition; + private final long metricReporterTopicRetentionTime; + private final String metricReporterTopicCleanupPolicy; + private final long consumerPollTimeout; + private final String consumerClientIdPrefix; + private final String consumerGroupIdPrefix; + private final ClusterModel clusterModel; + private final KafkaThread retrieveTask; + private final Lock lock; + private final Condition cond; + private final Controller controller; + private final ScheduledExecutorService executorService; + private final Set brokerIdsInUse; + private volatile boolean leaderEpochInitialized; + private volatile boolean isLeader; + private volatile Consumer consumer; + private volatile boolean shutdown; + + public LoadRetriever(AutoBalancerControllerConfig config, Controller controller, ClusterModel clusterModel) { + this(config, controller, clusterModel, null); + } + + public LoadRetriever(AutoBalancerControllerConfig config, Controller controller, ClusterModel clusterModel, LogContext logContext) { + if (logContext == null) { + logContext = new LogContext("[LoadRetriever] "); + } + this.logger = logContext.logger(LoadRetriever.class); + this.controller = controller; + this.clusterModel = clusterModel; + this.bootstrapServerMap = new HashMap<>(); + this.brokerIdsInUse = new HashSet<>(); + this.lock = new ReentrantLock(); + this.cond = lock.newCondition(); + this.executorService = Executors.newSingleThreadScheduledExecutor(new AutoBalancerThreadFactory("metric-topic-initializer")); + leaderEpochInitialized = false; + metricReporterTopic = config.getString(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG); + metricReporterTopicPartition = config.getInt(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG); + metricReporterTopicRetentionTime = config.getLong(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS_CONFIG); + metricReporterTopicCleanupPolicy = config.getString(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY); + consumerPollTimeout = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT); + consumerClientIdPrefix = config.getString(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX); + consumerGroupIdPrefix = config.getString(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX); + retrieveTask = KafkaThread.daemon("retrieve-load-task", this); + } + + public void start() { + this.shutdown = false; + this.executorService.scheduleAtFixedRate(this::checkAndCreateTopic, 0, 1L, TimeUnit.MINUTES); + retrieveTask.start(); + logger.info("Started"); + } + + public void shutdown() { + this.shutdown = true; + this.executorService.shutdown(); + retrieveTask.interrupt(); + logger.info("Shutdown completed"); + } + + private KafkaConsumer createConsumer(String bootstrapServer) { + long randomToken = RANDOM.nextLong(); + Properties consumerProps = new Properties(); + consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); + consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientIdPrefix + "-consumer-" + randomToken); + consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupIdPrefix + "-group-" + randomToken); + consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + consumerProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, MetricSerde.class.getName()); + return new KafkaConsumer<>(consumerProps); + } + + static class BrokerEndpoints { + private final int brokerId; + private Set endpoints = new HashSet<>(); + + private boolean isFenced; + + public BrokerEndpoints(int brokerId) { + this.brokerId = brokerId; + } + + public int brokerId() { + return this.brokerId; + } + + public Set getEndpoints() { + return this.endpoints; + } + + public void setEndpoints(Set endpoints) { + this.endpoints = new HashSet<>(endpoints); + } + + public BrokerEndpoints setFenced(boolean isFenced) { + this.isFenced = isFenced; + return this; + } + + public boolean isFenced() { + return this.isFenced; + } + + } + + @Override + public void onBrokerRegister(RegisterBrokerRecord record) { + lock.lock(); + try { + boolean isFenced = record.fenced() || record.inControlledShutdown(); + Set endpoints = new HashSet<>(); + for (RegisterBrokerRecord.BrokerEndpoint endpoint : record.endPoints()) { + String url = endpoint.host() + ":" + endpoint.port(); + endpoints.add(url); + } + BrokerEndpoints brokerEndpoints = new BrokerEndpoints(record.brokerId()); + brokerEndpoints.setFenced(isFenced); + brokerEndpoints.setEndpoints(endpoints); + this.bootstrapServerMap.put(record.brokerId(), brokerEndpoints); + cond.signal(); + } finally { + lock.unlock(); + } + } + + @Override + public void onBrokerUnregister(UnregisterBrokerRecord record) { + lock.lock(); + try { + this.bootstrapServerMap.remove(record.brokerId()); + } finally { + lock.unlock(); + } + + } + + @Override + public void onBrokerRegistrationChanged(BrokerRegistrationChangeRecord record) { + boolean isFenced = record.fenced() == BrokerRegistrationFencingChange.FENCE.value() + || record.inControlledShutdown() == BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value(); + lock.lock(); + try { + BrokerEndpoints brokerEndpoints = this.bootstrapServerMap.get(record.brokerId()); + if (brokerEndpoints != null) { + brokerEndpoints.setFenced(isFenced); + } + cond.signal(); + } finally { + lock.unlock(); + } + } + + private boolean hasAvailableBrokerInUse() { + if (brokerIdsInUse.isEmpty()) { + return false; + } + for (int brokerId : brokerIdsInUse) { + BrokerEndpoints brokerEndpoints = this.bootstrapServerMap.get(brokerId); + if (brokerEndpoints != null) { + if (!brokerEndpoints.isFenced && !brokerEndpoints.getEndpoints().isEmpty()) { + return true; + } + } + } + return false; + } + + private boolean hasAvailableBroker() { + if (this.bootstrapServerMap.isEmpty()) { + return false; + } + for (BrokerEndpoints brokerEndpoints : this.bootstrapServerMap.values()) { + if (!brokerEndpoints.isFenced() && !brokerEndpoints.getEndpoints().isEmpty()) { + return true; + } + } + return false; + } + + public String buildBootstrapServer() { + Set endpoints = new HashSet<>(); + this.brokerIdsInUse.clear(); + for (BrokerEndpoints brokerEndpoints : this.bootstrapServerMap.values()) { + if (!brokerEndpoints.isFenced() && !brokerEndpoints.getEndpoints().isEmpty()) { + endpoints.add(brokerEndpoints.getEndpoints().iterator().next()); + this.brokerIdsInUse.add(brokerEndpoints.brokerId()); + } + } + return String.join(",", endpoints); + } + + private void checkAndCreateConsumer() { + String bootstrapServer = ""; + this.lock.lock(); + try { + if (!hasAvailableBrokerInUse()) { + if (this.consumer != null) { + this.consumer.close(); + this.consumer = null; + logger.warn("No available broker found, close consumer"); + } + while (!shutdown && !hasAvailableBroker()) { + try { + this.cond.await(); + } catch (InterruptedException ignored) { + + } + } + if (this.shutdown) { + return; + } + bootstrapServer = buildBootstrapServer(); + } + } finally { + lock.unlock(); + } + if (this.consumer == null && !bootstrapServer.isEmpty()) { + //TODO: fetch metadata from controller + this.consumer = createConsumer(bootstrapServer); + this.consumer.subscribe(Collections.singleton(metricReporterTopic)); + logger.info("Created consumer on {}", bootstrapServer); + } + } + + private void checkAndCreateTopic() { + //TODO: check with cache + if (!leaderEpochInitialized || !isLeader) { + return; + } + + CreateTopicsRequestData request = new CreateTopicsRequestData(); + CreateTopicsRequestData.CreatableTopicCollection topicCollection = new CreateTopicsRequestData.CreatableTopicCollection(); + CreateTopicsRequestData.CreateableTopicConfigCollection configCollection = new CreateTopicsRequestData.CreateableTopicConfigCollection(); + configCollection.add(new CreateTopicsRequestData.CreateableTopicConfig() + .setName(TopicConfig.RETENTION_MS_CONFIG) + .setValue(Long.toString(metricReporterTopicRetentionTime))); + configCollection.add(new CreateTopicsRequestData.CreateableTopicConfig() + .setName(TopicConfig.CLEANUP_POLICY_CONFIG) + .setValue(metricReporterTopicCleanupPolicy)); + + topicCollection.add(new CreateTopicsRequestData.CreatableTopic() + .setName(metricReporterTopic) + .setNumPartitions(metricReporterTopicPartition) + .setReplicationFactor((short) 1) + .setConfigs(configCollection)); + request.setTopics(topicCollection); + CompletableFuture future = this.controller.createTopics( + new ControllerRequestContext(null, null, OptionalLong.empty()), + request, + Collections.emptySet()); + try { + CreateTopicsResponseData rsp = future.get(); + CreateTopicsResponseData.CreatableTopicResult result = rsp.topics().find(metricReporterTopic); + if (result.errorCode() == Errors.NONE.code()) { + logger.info("Create metrics reporter topic {} succeed", metricReporterTopic); + } + if (result.errorCode() != Errors.NONE.code() && result.errorCode() != Errors.TOPIC_ALREADY_EXISTS.code()) { + logger.warn("Create metrics reporter topic {} failed: {}", metricReporterTopic, result.errorMessage()); + } + } catch (Exception e) { + logger.error("Create metrics reporter topic {} exception: {}", metricReporterTopic, e.getMessage()); + } + } + + @Override + public void run() { + while (!shutdown) { + checkAndCreateConsumer(); + if (shutdown) { + break; + } + try { + ConsumerRecords records = this.consumer.poll(Duration.ofMillis(consumerPollTimeout)); + for (ConsumerRecord record : records) { + if (record == null) { + // This means we cannot parse the metrics. It might happen when a newer type of metrics has been added and + // the current code is still old. We simply ignore that metric in this case. + logger.warn("Cannot parse record, maybe controller version is outdated."); + continue; + } + updateClusterModel(record.value()); + } + logger.debug("Finished consuming {} metrics from {}.", records.count(), metricReporterTopic); + } catch (Exception e) { + logger.error("Consumer poll error: {}", e.getMessage()); + } + } + if (this.consumer != null) { + try { + this.consumer.close(Duration.ofMillis(5000)); + } catch (Exception e) { + logger.error("Exception when close consumer: {}", e.getMessage()); + } + } + } + + public void onLeaderChanged(boolean isLeader) { + this.leaderEpochInitialized = true; + this.isLeader = isLeader; + } + + private void updateClusterModel(AutoBalancerMetrics metrics) { + switch (metrics.metricClassId()) { + case BROKER_METRIC: + clusterModel.updateBroker((BrokerMetrics) metrics); + break; + case PARTITION_METRIC: + clusterModel.updateTopicPartition((TopicPartitionMetrics) metrics); + 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 new file mode 100644 index 0000000000..01860089f9 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/Action.java @@ -0,0 +1,81 @@ +/* + * 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; + +import org.apache.kafka.common.TopicPartition; + +public class Action { + private final TopicPartition srcTp; + private final TopicPartition destTp; + private final int srcBrokerId; + private final int destBrokerId; + + private final ActionType type; + + public Action(ActionType type, TopicPartition srcTp, int srcBrokerId, int destBrokerId) { + this(type, srcTp, srcBrokerId, destBrokerId, null); + } + + public Action(ActionType type, TopicPartition srcTp, int srcBrokerId, int destBrokerId, TopicPartition destTp) { + this.type = type; + this.srcTp = srcTp; + this.srcBrokerId = srcBrokerId; + this.destBrokerId = destBrokerId; + this.destTp = destTp; + } + + public ActionType getType() { + return type; + } + + public TopicPartition getSrcTopicPartition() { + return srcTp; + } + + public TopicPartition getDestTopicPartition() { + return destTp; + } + + public int getDestBrokerId() { + return destBrokerId; + } + + public int getSrcBrokerId() { + return srcBrokerId; + } + + @Override + public String toString() { + return "Action{" + + "srcTp=" + srcTp + + ", destTp=" + destTp + + ", srcBrokerId=" + srcBrokerId + + ", destBrokerId=" + destBrokerId + + ", type=" + type + + '}'; + } + + public String prettyString() { + if (this.type == ActionType.MOVE) { + return String.format("Action-%s: %s@node-%d ---> node-%d", type, srcTp, srcBrokerId, destBrokerId); + } else if (this.type == ActionType.SWAP) { + return String.format("Action-%s: %s@node-%d <--> %s@node-%d", type, srcTp, srcBrokerId, destTp, destBrokerId); + } + return toString(); + } +} diff --git a/core/src/main/java/kafka/autobalancer/common/ActionType.java b/core/src/main/java/kafka/autobalancer/common/ActionType.java new file mode 100644 index 0000000000..ab028f7555 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/ActionType.java @@ -0,0 +1,23 @@ +/* + * 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; + +public enum ActionType { + MOVE, + SWAP +} diff --git a/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java b/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.java new file mode 100644 index 0000000000..6eceb7c982 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/AutoBalancerThreadFactory.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; + +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 final String name; + private final boolean daemon; + private final AtomicInteger id = new AtomicInteger(0); + private final Logger logger; + + public AutoBalancerThreadFactory(String name) { + this(name, true, null); + } + + public AutoBalancerThreadFactory(String name, boolean daemon, Logger logger) { + this.name = name; + this.daemon = daemon; + this.logger = logger == null ? LOGGER : logger; + } + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(r, name + "-" + id.getAndIncrement()); + t.setDaemon(daemon); + t.setUncaughtExceptionHandler((t1, e) -> logger.error("Uncaught exception in " + t1.getName() + ": ", e)); + return t; + } +} diff --git a/core/src/main/java/kafka/autobalancer/common/Resource.java b/core/src/main/java/kafka/autobalancer/common/Resource.java new file mode 100644 index 0000000000..21c222ea2f --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/common/Resource.java @@ -0,0 +1,91 @@ +/* + * 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. + */ +/* + * Some portion of this fil 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.common; + +import java.util.List; + + +/** + * CPU: a host and broker-level resource. + * NW (in and out): a host-level resource. + * DISK: a broker-level resource. + */ +public enum Resource { + CPU("cpu", 0, 0.001), + NW_IN("networkInbound", 1, 10), + NW_OUT("networkOutbound", 2, 10); + + // 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 + // more than 0.1% of sum value. + private static final double EPSILON_PERCENT = 0.0008; + private static final List CACHED_VALUES = List.of(values()); + private final String resource; + private final int id; + private final double epsilon; + + Resource(String resource, int id, double epsilon) { + this.resource = resource; + this.id = id; + this.epsilon = epsilon; + } + + /** + * Use this instead of values() because values() creates a new array each time. + * + * @return enumerated values in the same order as values() + */ + public static List cachedValues() { + return CACHED_VALUES; + } + + /** + * @return The resource type. + */ + public String resource() { + return resource; + } + + /** + * @return The resource id. + */ + public int id() { + return id; + } + + /** + * The epsilon value used in comparing the given values. + * + * @param value1 The first value used in comparison. + * @param value2 The second value used in comparison. + * @return The epsilon value used in comparing the given values. + */ + public double epsilon(double value1, double value2) { + return Math.max(epsilon, EPSILON_PERCENT * (value1 + value2)); + } + + @Override + public String toString() { + return resource; + } +} + diff --git a/core/src/main/java/kafka/autobalancer/config/AutoBalancerConfig.java b/core/src/main/java/kafka/autobalancer/config/AutoBalancerConfig.java new file mode 100644 index 0000000000..2dcf3794de --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/config/AutoBalancerConfig.java @@ -0,0 +1,76 @@ +/* + * 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.config; + +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.TopicConfig; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class AutoBalancerConfig extends AbstractConfig { + protected static final ConfigDef CONFIG; + private static final String PREFIX = "autobalancer"; + + /* Configurations */ + public static final String AUTO_BALANCER_TOPIC_CONFIG = PREFIX + "topic"; + public static final String AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG = PREFIX + "topic.num.partitions"; + public static final String AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS_CONFIG = PREFIX + "topic.retention.ms"; + public static final String AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY = PREFIX + "topic.cleanup.policy"; + /* Default values */ + public static final String DEFAULT_AUTO_BALANCER_TOPIC = "__auto_balancer_metrics"; + public static final Integer DEFAULT_AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS = -1; + public static final long DEFAULT_AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS = TimeUnit.HOURS.toMillis(5); + public static final String DEFAULT_AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY = String.join(",", + TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE); + /* Documents */ + private static final String AUTO_BALANCER_TOPIC_DOC = "The topic to which Auto Balancer metrics reporter " + + "should send messages"; + private static final String AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_DOC = "The number of partitions of Auto Balancer metrics topic"; + private static final String AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS_DOC = TopicConfig.RETENTION_MS_DOC; + public static final String AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY_DOC = TopicConfig.CLEANUP_POLICY_DOC; + + static { + CONFIG = new ConfigDef() + .define(AUTO_BALANCER_TOPIC_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_AUTO_BALANCER_TOPIC, + ConfigDef.Importance.HIGH, + AUTO_BALANCER_TOPIC_DOC) + .define(AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG, + ConfigDef.Type.INT, + DEFAULT_AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_DOC) + .define(AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS_CONFIG, + ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_TOPIC_RETENTION_MS_DOC) + .define(AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY, + ConfigDef.Type.STRING, + DEFAULT_AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY, + ConfigDef.Importance.HIGH, + AUTO_BALANCER_METRICS_TOPIC_CLEANUP_POLICY_DOC); + } + + public AutoBalancerConfig(Map originals, boolean doLogs) { + super(CONFIG, originals, doLogs); + } +} diff --git a/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java b/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java new file mode 100644 index 0000000000..9b639f4ed8 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/config/AutoBalancerControllerConfig.java @@ -0,0 +1,149 @@ +/* + * 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.config; + +import kafka.autobalancer.goals.NetworkInCapacityGoal; +import kafka.autobalancer.goals.NetworkInDistributionGoal; +import kafka.autobalancer.goals.NetworkOutCapacityGoal; +import kafka.autobalancer.goals.NetworkOutDistributionGoal; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.ConfigDef; + +import java.time.Duration; +import java.util.Map; +import java.util.StringJoiner; + +public class AutoBalancerControllerConfig extends AutoBalancerConfig { + /* Configurations */ + private static final String PREFIX = "autobalancer.controller."; + public static final String AUTO_BALANCER_CONTROLLER_ENABLE = PREFIX + "enable"; + public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT = PREFIX + "consumer.poll.timeout"; + public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX = PREFIX + "consumer.client.id"; + public static final String AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX = PREFIX + CommonClientConfigs.GROUP_ID_CONFIG; + 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_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_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_LOAD_AGGREGATION = PREFIX + "load.aggregation"; + public static final String AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS = PREFIX + "exclude.broker.ids"; + public static final String AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS = PREFIX + "exclude.topics"; + /* Default values */ + public static final boolean DEFAULT_AUTO_BALANCER_CONTROLLER_ENABLE = false; + public static final long DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT = 1000L; + public static final String DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX = "AutoBalancerControllerConsumer"; + public static final String DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX = "AutoBalancerControllerConsumerGroup"; + 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(); + 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 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 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 = 100; + public static final boolean DEFAULT_AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION = false; + public static final String DEFAULT_AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS = ""; + public static final String DEFAULT_AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS = ""; + /* Documents */ + 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_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_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_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_LOAD_AGGREGATION_DOC = "Use aggregation of partition load as broker load, instead of using reported broker metrics directly"; + public static final String AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS_DOC = "Broker ids that auto balancer will ignore during balancing, separated by comma"; + public static final String AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS_DOC = "Topics that auto balancer will ignore during balancing, separated by comma"; + + static { + CONFIG.define(AUTO_BALANCER_CONTROLLER_ENABLE, ConfigDef.Type.BOOLEAN, + DEFAULT_AUTO_BALANCER_CONTROLLER_ENABLE, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_ENABLE_DOC) + .define(AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT, ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT_DOC) + .define(AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX, ConfigDef.Type.STRING, + DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX, ConfigDef.Importance.LOW, + AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX_DOC) + .define(AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX, ConfigDef.Type.STRING, + DEFAULT_AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_CONSUMER_GROUP_ID_PREFIX_DOC) + .define(AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS, ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS_DOC) + .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_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_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) + .define(AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS, ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_ANOMALY_DETECT_INTERVAL_MS_DOC) + .define(AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION, ConfigDef.Type.BOOLEAN, + DEFAULT_AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION_DOC) + .define(AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS, ConfigDef.Type.LIST, + DEFAULT_AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_EXCLUDE_BROKER_IDS_DOC) + .define(AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS, ConfigDef.Type.LIST, + DEFAULT_AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS, ConfigDef.Importance.HIGH, + AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS_DOC); + } + + public AutoBalancerControllerConfig(Map originals, boolean doLog) { + super(originals, doLog); + } +} diff --git a/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java b/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java new file mode 100644 index 0000000000..2cb0ae0eeb --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/config/AutoBalancerMetricsReporterConfig.java @@ -0,0 +1,156 @@ +/* + * 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.config; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.ConfigDef; + +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsReporterConfig. + */ +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"; + public static final String AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_CONFIG = PREFIX + "metrics.reporting.interval.ms"; + 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 = 10000.0; + public static final double DEFAULT_AUTO_BALANCER_BROKER_NW_OUT_CAPACITY = 10000.0; + 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; + public static final Short DEFAULT_AUTO_BALANCER_METRICS_TOPIC_REPLICATION_FACTOR = 1; + public static final long DEFAULT_AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS = TimeUnit.MINUTES.toMillis(1); + 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 KB/s"; + public static final String AUTO_BALANCER_BROKER_NW_OUT_CAPACITY_DOC = "Maximum network output bandwidth available for the broker in KB/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"; + private static final String AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES_DOC = "Number of times the Auto Balancer metrics reporter will " + + "attempt to create the producer while starting up."; + private static final String AUTO_BALANCER_METRICS_REPORTER_INTERVAL_MS_DOC = "The interval in milliseconds the " + + "metrics reporter should report the metrics."; + public static final String AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID_DOC = CommonClientConfigs.CLIENT_ID_DOC; + private static final String AUTO_BALANCER_METRICS_REPORTER_LINGER_MS_DOC = "The linger.ms configuration of KafkaProducer used in Cruise " + + "Control 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 Cruise " + + "Control 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, + 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, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_TIMEOUT_MS_DOC) + .define(AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES_CONFIG, + ConfigDef.Type.INT, + DEFAULT_AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_TOPIC_AUTO_CREATE_RETRIES_DOC) + .define(AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES_CONFIG, + ConfigDef.Type.INT, + DEFAULT_AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES_DOC) + .define(AUTO_BALANCER_METRICS_REPORTER_LINGER_MS_CONFIG, + ConfigDef.Type.LONG, + DEFAULT_AUTO_BALANCER_METRICS_REPORTER_LINGER_MS, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_REPORTER_LINGER_MS_DOC) + .define(AUTO_BALANCER_METRICS_REPORTER_BATCH_SIZE_CONFIG, + ConfigDef.Type.INT, + DEFAULT_AUTO_BALANCER_METRICS_BATCH_SIZE, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_REPORTER_BATCH_SIZE_DOC) + .define(AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID, + ConfigDef.Type.STRING, + DEFAULT_AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID, + ConfigDef.Importance.LOW, + AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID_DOC); + } + + public AutoBalancerMetricsReporterConfig(Map originals, boolean doLog) { + super(originals, doLog); + } + + /** + * @param baseConfigName Base config name. + * @return Auto balancer metrics reporter config name. + */ + public static String config(String baseConfigName) { + String configName = PREFIX + baseConfigName; + if (!CONFIGS.contains(configName)) { + throw new IllegalArgumentException("The base config name " + baseConfigName + " is not defined."); + } + return configName; + } + + public static Properties parseProducerConfigs(Map configMap) { + Properties props = new Properties(); + for (Map.Entry entry : configMap.entrySet()) { + if (entry.getKey().startsWith(PREFIX)) { + props.put(entry.getKey().replace(PREFIX, ""), entry.getValue()); + } + } + return props; + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java new file mode 100644 index 0000000000..8673b57e06 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractGoal.java @@ -0,0 +1,148 @@ +/* + * 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.model.BrokerUpdater.Broker; +import kafka.autobalancer.model.ClusterModelSnapshot; +import kafka.autobalancer.model.ModelUtils; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater.TopicPartitionReplica; +import org.apache.kafka.common.Configurable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +public abstract class AbstractGoal implements Goal, Configurable, Comparable { + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractGoal.class); + protected static final double POSITIVE_ACTION_SCORE_THRESHOLD = 0.5; + + /** + * Calculate the score difference of src and dest. The score should be normalized to [0, 1.0] + * + * @param srcBrokerBefore source broker before action + * @param destBrokerBefore dest broker before action + * @param srcBrokerAfter source broker after action + * @param destBrokerAfter dest broker after action + * @return normalized score. < 0.5 means negative action + * == 0.5 means action with no affection + * > 0.5 means positive action + */ + private double scoreDelta(Broker srcBrokerBefore, Broker destBrokerBefore, Broker srcBrokerAfter, 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); + } + + /** + * Calculate acceptance score based on status change of src and dest brokers. + * + * @param srcBrokerBefore source broker before action + * @param destBrokerBefore dest broker before action + * @param srcBrokerAfter source broker after action + * @param destBrokerAfter dest broker after action + * @return normalized score. 0 means not allowed action + * > 0 means permitted action, but can be positive or negative for this goal + */ + private double calculateAcceptanceScore(Broker srcBrokerBefore, Broker destBrokerBefore, Broker srcBrokerAfter, Broker destBrokerAfter) { + double score = scoreDelta(srcBrokerBefore, destBrokerBefore, srcBrokerAfter, destBrokerAfter); + boolean isSrcBrokerAcceptedBefore = isBrokerAcceptable(srcBrokerBefore); + boolean isDestBrokerAcceptedBefore = isBrokerAcceptable(destBrokerBefore); + boolean isSrcBrokerAcceptedAfter = isBrokerAcceptable(srcBrokerAfter); + boolean isDestBrokerAcceptedAfter = isBrokerAcceptable(destBrokerAfter); + + if (!isHardGoal()) { + return score; + } + + if (isSrcBrokerAcceptedBefore && !isSrcBrokerAcceptedAfter) { + return 0.0; + } else if (isDestBrokerAcceptedBefore && !isDestBrokerAcceptedAfter) { + return 0.0; + } + + if (!isSrcBrokerAcceptedBefore && !isSrcBrokerAcceptedAfter) { + return score <= POSITIVE_ACTION_SCORE_THRESHOLD ? 0.0 : score; + } else if (!isDestBrokerAcceptedBefore && !isDestBrokerAcceptedAfter) { + return score <= POSITIVE_ACTION_SCORE_THRESHOLD ? 0.0 : score; + } + return score; + } + + @Override + 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()); + + switch (action.getType()) { + case MOVE: + ModelUtils.moveReplicaLoad(srcBrokerAfter, destBrokerAfter, srcReplica); + break; + case SWAP: + ModelUtils.moveReplicaLoad(srcBrokerAfter, destBrokerAfter, srcReplica); + ModelUtils.moveReplicaLoad(destBrokerAfter, srcBrokerAfter, + cluster.replica(action.getDestBrokerId(), action.getDestTopicPartition())); + break; + default: + return 0.0; + } + + return calculateAcceptanceScore(srcBrokerBefore, destBrokerBefore, srcBrokerAfter, destBrokerAfter); + } + + @Override + public int priority() { + return GoalUtils.priority(this); + } + + @Override + public Set getEligibleBrokers(ClusterModelSnapshot cluster) { + return cluster.brokers().stream().filter(Broker::isActive).collect(Collectors.toSet()); + } + + @Override + public int compareTo(AbstractGoal other) { + return Integer.compare(other.priority(), this.priority()); + } + + @Override + public int hashCode() { + return Objects.hashCode(name()); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AbstractGoal goal = (AbstractGoal) o; + return name().equals(goal.name()); + } + + @Override + public String toString() { + return name(); + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java new file mode 100644 index 0000000000..c16e638388 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceCapacityGoal.java @@ -0,0 +1,89 @@ +/* + * 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.BrokerUpdater.Broker; +import kafka.autobalancer.model.ClusterModelSnapshot; +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 new file mode 100644 index 0000000000..948ec724a1 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceDistributionGoal.java @@ -0,0 +1,127 @@ +/* + * 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.model.BrokerUpdater; +import kafka.autobalancer.model.ClusterModelSnapshot; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +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); + } + + @Override + public List optimize(ClusterModelSnapshot cluster, Collection goalsByPriority) { + List actions = new ArrayList<>(); + validateConfig(); + Set eligibleBrokers = getEligibleBrokers(cluster); + calcUtilizationBound(eligibleBrokers); + List brokersToOptimize = new ArrayList<>(); + for (BrokerUpdater.Broker broker : eligibleBrokers) { + if (!isBrokerAcceptable(broker)) { + LOGGER.warn("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)) { + 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)) { + List brokerActions = tryIncreaseLoadByAction(ActionType.MOVE, cluster, broker, candidateBrokers, goalsByPriority); + if (isBrokerAcceptable(broker)) { + brokerActions.addAll(tryIncreaseLoadByAction(ActionType.SWAP, cluster, broker, candidateBrokers, goalsByPriority)); + } + actions.addAll(brokerActions); + } + + if (!isBrokerAcceptable(broker)) { + // broker still violates goal after iterating all partitions + onBalanceFailed(broker); + } + } + 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; + } + + private boolean requireMoreLoad(double util) { + return util < resourceUtilDistLowerBound; + } + + @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); + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java new file mode 100644 index 0000000000..afaceb2634 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/AbstractResourceGoal.java @@ -0,0 +1,207 @@ +/* + * 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.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; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public abstract class AbstractResourceGoal extends AbstractGoal { + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractGoal.class); + + abstract Resource resource(); + + private Optional getAcceptableAction(List> candidateActionScores) { + Action acceptableAction = null; + Optional> optionalEntry = candidateActionScores.stream() + .max(Comparator.comparingDouble(Map.Entry::getValue)); + if (optionalEntry.isPresent() && optionalEntry.get().getValue() > POSITIVE_ACTION_SCORE_THRESHOLD) { + acceptableAction = optionalEntry.get().getKey(); + } + return Optional.ofNullable(acceptableAction); + } + + private double normalizeGoalsScore(Map scoreMap) { + int totalWeight = scoreMap.keySet().stream().mapToInt(AbstractGoal::priority).sum(); + return scoreMap.entrySet().stream() + .mapToDouble(entry -> entry.getValue() * (double) entry.getKey().priority() / totalWeight) + .sum(); + } + + private Optional trySwapPartitionOut(ClusterModelSnapshot cluster, + TopicPartitionReplicaUpdater.TopicPartitionReplica srcReplica, + BrokerUpdater.Broker srcBroker, + List candidates, + Collection goalsByPriority) { + List> candidateActionScores = new ArrayList<>(); + for (BrokerUpdater.Broker candidate : candidates) { + for (TopicPartitionReplicaUpdater.TopicPartitionReplica candidateReplica : cluster.replicasFor(candidate.getBrokerId())) { + if (candidate.load(resource()) > srcReplica.load(resource())) { + continue; + } + boolean isHardGoalViolated = false; + Action action = new Action(ActionType.SWAP, srcReplica.getTopicPartition(), srcBroker.getBrokerId(), + candidate.getBrokerId(), candidateReplica.getTopicPartition()); + Map scoreMap = new HashMap<>(); + for (AbstractGoal goal : goalsByPriority) { + double score = goal.actionAcceptanceScore(action, cluster); + if (goal.isHardGoal() && score == 0) { + isHardGoalViolated = true; + break; + } + scoreMap.put(goal, score); + } + if (!isHardGoalViolated) { + candidateActionScores.add(new AbstractMap.SimpleEntry<>(action, normalizeGoalsScore(scoreMap))); + } + } + } + LOGGER.debug("All possible action score: {}", candidateActionScores); + return getAcceptableAction(candidateActionScores); + } + + private Optional tryMovePartitionOut(ClusterModelSnapshot cluster, + TopicPartitionReplicaUpdater.TopicPartitionReplica replica, + BrokerUpdater.Broker srcBroker, + List candidates, + Collection goalsByPriority) { + List> candidateActionScores = new ArrayList<>(); + for (BrokerUpdater.Broker candidate : candidates) { + boolean isHardGoalViolated = false; + Action action = new Action(ActionType.MOVE, replica.getTopicPartition(), srcBroker.getBrokerId(), candidate.getBrokerId()); + Map scoreMap = new HashMap<>(); + for (AbstractGoal goal : goalsByPriority) { + double score = goal.actionAcceptanceScore(action, cluster); + if (goal.isHardGoal() && score == 0) { + isHardGoalViolated = true; + break; + } + scoreMap.put(goal, score); + } + if (isHardGoalViolated) { + break; + } + candidateActionScores.add(new AbstractMap.SimpleEntry<>(action, normalizeGoalsScore(scoreMap))); + } + LOGGER.debug("All possible action score: {} for {}", candidateActionScores, name()); + return getAcceptableAction(candidateActionScores); + } + + /** + * Try to reduce resource load by move or swap replicas out. + * + * @param actionType type of action + * @param cluster cluster model + * @param srcBroker broker to reduce load + * @param candidateBrokers candidate brokers to move replicas to, or swap replicas with + * @param goalsByPriority all configured goals sorted by priority + * @return a list of actions able to reduce load of srcBroker + */ + protected List tryReduceLoadByAction(ActionType actionType, + ClusterModelSnapshot cluster, + BrokerUpdater.Broker srcBroker, + List candidateBrokers, + Collection goalsByPriority) { + List actionList = new ArrayList<>(); + List srcReplicas = cluster + .replicasFor(srcBroker.getBrokerId()) + .stream() + .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 + Optional optionalAction; + if (actionType == ActionType.MOVE) { + optionalAction = tryMovePartitionOut(cluster, tp, srcBroker, candidateBrokers, goalsByPriority); + } else { + optionalAction = trySwapPartitionOut(cluster, tp, srcBroker, candidateBrokers, goalsByPriority); + } + + if (optionalAction.isPresent()) { + Action action = optionalAction.get(); + cluster.applyAction(action); + actionList.add(action); + } + if (isBrokerAcceptable(srcBroker)) { + // broker is acceptable after action, skip iterating reset partitions + return actionList; + } + } + return actionList; + } + + /** + * Try to increase resource load by move or swap replicas in. + * + * @param actionType type of action + * @param cluster cluster model + * @param srcBroker broker to increase load + * @param candidateBrokers candidate brokers to move replicas from, or swap replicas with + * @param goalsByPriority all configured goals sorted by priority + * @return a list of actions able to increase load of srcBroker + */ + protected List tryIncreaseLoadByAction(ActionType actionType, + ClusterModelSnapshot cluster, + BrokerUpdater.Broker srcBroker, + List candidateBrokers, + Collection goalsByPriority) { + List actionList = new ArrayList<>(); + candidateBrokers.sort(Comparator.comparingDouble(b -> -b.utilizationFor(resource()))); // higher load first + for (BrokerUpdater.Broker candidateBroker : candidateBrokers) { + List candidateReplicas = cluster + .replicasFor(candidateBroker.getBrokerId()) + .stream() + .sorted(Comparator.comparingDouble(r -> -r.load(resource()))) // higher load first + .collect(Collectors.toList()); + for (TopicPartitionReplicaUpdater.TopicPartitionReplica tp : candidateReplicas) { + Optional optionalAction; + if (actionType == ActionType.MOVE) { + optionalAction = tryMovePartitionOut(cluster, tp, candidateBroker, List.of(srcBroker), goalsByPriority); + } else { + optionalAction = trySwapPartitionOut(cluster, tp, candidateBroker, List.of(srcBroker), goalsByPriority); + } + + if (optionalAction.isPresent()) { + Action action = optionalAction.get(); + cluster.applyAction(action); + actionList.add(action); + } + if (isBrokerAcceptable(srcBroker)) { + // broker is acceptable after action, skip iterating reset partitions + return actionList; + } + } + } + return actionList; + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/Goal.java b/core/src/main/java/kafka/autobalancer/goals/Goal.java new file mode 100644 index 0000000000..4b4c3ee0d0 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/Goal.java @@ -0,0 +1,56 @@ +/* + * 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.model.BrokerUpdater; +import kafka.autobalancer.model.ClusterModelSnapshot; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +public interface Goal { + + boolean isHardGoal(); + + List optimize(ClusterModelSnapshot cluster, Collection goalsByPriority); + + void validateConfig(); + + void onBalanceFailed(BrokerUpdater.Broker broker); + + boolean isBrokerAcceptable(BrokerUpdater.Broker broker); + + int priority(); + + Set getEligibleBrokers(ClusterModelSnapshot cluster); + + String name(); + + double brokerScore(BrokerUpdater.Broker broker); + + /** + * Get the acceptance score of the goal if the action applied to the given cluster. + * + * @param action action to apply to the cluster + * @param cluster cluster to apply the action + * @return action acceptance score, 0 for not accepted + */ + double actionAcceptanceScore(Action action, ClusterModelSnapshot cluster); +} diff --git a/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java b/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java new file mode 100644 index 0000000000..ee1c457fad --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/GoalUtils.java @@ -0,0 +1,64 @@ +/* + * 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 java.util.HashMap; +import java.util.Map; + +public class GoalUtils { + private static final Map GOALS_PRIORITY_MAP = new HashMap<>(); + + static { + GOALS_PRIORITY_MAP.put(NetworkInCapacityGoal.class.getSimpleName(), 10); + GOALS_PRIORITY_MAP.put(NetworkOutCapacityGoal.class.getSimpleName(), 10); + GOALS_PRIORITY_MAP.put(NetworkInDistributionGoal.class.getSimpleName(), 8); + GOALS_PRIORITY_MAP.put(NetworkOutDistributionGoal.class.getSimpleName(), 8); + } + + public static int priority(AbstractGoal goal) { + return GOALS_PRIORITY_MAP.getOrDefault(goal.name(), 0); + } + + public static boolean isValidAction(Action action, ClusterModelSnapshot cluster) { + if (cluster.broker(action.getSrcBrokerId()) == null + || cluster.broker(action.getDestBrokerId()) == null + || cluster.replica(action.getSrcBrokerId(), action.getSrcTopicPartition()) == null) { + return false; + } + if (action.getType() == ActionType.SWAP) { + return action.getDestTopicPartition() != null + && cluster.replica(action.getDestBrokerId(), action.getDestTopicPartition()) != null; + } + return true; + } + + public static double normalize(double value, double max, double min) { + return normalize(value, max, min, false); + } + + public static double normalize(double value, double max, double min, boolean reverse) { + if (reverse) { + 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 new file mode 100644 index 0000000000..a50f24d67b --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkInCapacityGoal.java @@ -0,0 +1,51 @@ +/* + * 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.config.AutoBalancerControllerConfig; +import kafka.autobalancer.model.BrokerUpdater; +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 + 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()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java new file mode 100644 index 0000000000..f1d4237a85 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkInDistributionGoal.java @@ -0,0 +1,52 @@ +/* + * 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.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); + + @Override + public String name() { + return NetworkInDistributionGoal.class.getSimpleName(); + } + + @Override + Resource resource() { + return Resource.NW_IN; + } + + @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); + } + + @Override + public void onBalanceFailed(BrokerUpdater.Broker broker) { + LOGGER.warn("Failed to balance broker {} network inbound load after iterating all partitions", broker.getBrokerId()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java new file mode 100644 index 0000000000..e9d08961df --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkOutCapacityGoal.java @@ -0,0 +1,51 @@ +/* + * 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.config.AutoBalancerControllerConfig; +import kafka.autobalancer.model.BrokerUpdater; +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 + 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()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java b/core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java new file mode 100644 index 0000000000..7376fbdd2e --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/goals/NetworkOutDistributionGoal.java @@ -0,0 +1,52 @@ +/* + * 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.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); + + @Override + public String name() { + return NetworkOutDistributionGoal.class.getSimpleName(); + } + + @Override + Resource resource() { + return Resource.NW_OUT; + } + + @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); + } + + @Override + public void onBalanceFailed(BrokerUpdater.Broker broker) { + LOGGER.warn("Failed to balance broker {} network outbound load after iterating all partitions", broker.getBrokerId()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/listeners/BrokerStatusListener.java b/core/src/main/java/kafka/autobalancer/listeners/BrokerStatusListener.java new file mode 100644 index 0000000000..e0c6730d92 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/listeners/BrokerStatusListener.java @@ -0,0 +1,30 @@ +/* + * 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.listeners; + +import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; + +public interface BrokerStatusListener { + void onBrokerRegister(RegisterBrokerRecord record); + + void onBrokerUnregister(UnregisterBrokerRecord record); + + void onBrokerRegistrationChanged(BrokerRegistrationChangeRecord record); +} diff --git a/core/src/main/java/kafka/autobalancer/listeners/ClusterStatusListenerRegistry.java b/core/src/main/java/kafka/autobalancer/listeners/ClusterStatusListenerRegistry.java new file mode 100644 index 0000000000..510a9d52c1 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/listeners/ClusterStatusListenerRegistry.java @@ -0,0 +1,42 @@ +/* + * 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.listeners; + +import java.util.ArrayList; +import java.util.List; + +public class ClusterStatusListenerRegistry { + private final List brokerListeners = new ArrayList<>(); + private final List topicPartitionListeners = new ArrayList<>(); + + public void register(BrokerStatusListener listener) { + brokerListeners.add(listener); + } + + public void register(TopicPartitionStatusListener listener) { + topicPartitionListeners.add(listener); + } + + public List brokerListeners() { + return brokerListeners; + } + + public List topicPartitionListeners() { + return topicPartitionListeners; + } +} diff --git a/core/src/main/java/kafka/autobalancer/listeners/LeaderChangeListener.java b/core/src/main/java/kafka/autobalancer/listeners/LeaderChangeListener.java new file mode 100644 index 0000000000..c623add4aa --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/listeners/LeaderChangeListener.java @@ -0,0 +1,24 @@ +/* + * 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.listeners; + +public interface LeaderChangeListener { + void onBecomeLeader(); + + void onResign(); +} diff --git a/core/src/main/java/kafka/autobalancer/listeners/TopicPartitionStatusListener.java b/core/src/main/java/kafka/autobalancer/listeners/TopicPartitionStatusListener.java new file mode 100644 index 0000000000..4e0d8dec85 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/listeners/TopicPartitionStatusListener.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.listeners; + +import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RemoveTopicRecord; +import org.apache.kafka.common.metadata.TopicRecord; + +public interface TopicPartitionStatusListener { + void onTopicCreate(TopicRecord record); + + void onTopicDelete(RemoveTopicRecord record); + + void onPartitionCreate(PartitionRecord record); + + void onPartitionChange(PartitionChangeRecord record); +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java new file mode 100644 index 0000000000..2ab90a7567 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporter.java @@ -0,0 +1,451 @@ +/* + * 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. + */ +/* + * Some portion of this file 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; + +import com.yammer.metrics.core.Metric; +import com.yammer.metrics.core.MetricName; +import com.yammer.metrics.core.MetricsRegistry; +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.MetricSerde; +import kafka.autobalancer.metricsreporter.metric.MetricsUtils; +import kafka.autobalancer.metricsreporter.metric.RawMetricType; +import kafka.autobalancer.metricsreporter.metric.YammerMetricProcessor; +import kafka.server.KafkaConfig; +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.KafkaThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsReporter. + */ +public class AutoBalancerMetricsReporter implements MetricsRegistryListener, MetricsReporter, Runnable { + public static final String DEFAULT_BOOTSTRAP_SERVERS_HOST = "localhost"; + public static final String DEFAULT_BOOTSTRAP_SERVERS_PORT = "9092"; + protected static final Duration PRODUCER_CLOSE_TIMEOUT = Duration.ofSeconds(5); + private static final Logger LOGGER = LoggerFactory.getLogger(AutoBalancerMetricsReporter.class); + // KafkaYammerMetrics class in Kafka 3.3+ + private static final String YAMMER_METRICS_IN_KAFKA_3_3_AND_LATER = "org.apache.kafka.server.metrics.KafkaYammerMetrics"; + // KafkaYammerMetrics class in Kafka 2.6+ + private static final String YAMMER_METRICS_IN_KAFKA_2_6_AND_LATER = "kafka.metrics.KafkaYammerMetrics"; + // KafkaYammerMetrics class in Kafka 2.5- + private static final String YAMMER_METRICS_IN_KAFKA_2_5_AND_EARLIER = "com.yammer.metrics.Metrics"; + private final Map interestedMetrics = new ConcurrentHashMap<>(); + private final MetricsRegistry metricsRegistry = metricsRegistry(); + private YammerMetricProcessor yammerMetricProcessor; + private KafkaThread metricsReporterRunner; + private KafkaProducer producer; + private String autoBalancerMetricsTopic; + private long reportingIntervalMs; + private int brokerId; + private String brokerRack; + private long lastReportingTime = System.currentTimeMillis(); + 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"); + String listeners = String.valueOf(configs.get(KafkaConfig.ListenersProp())); + if (!"null".equals(listeners) && listeners.length() != 0) { + // See https://kafka.apache.org/documentation/#listeners for possible responses. If multiple listeners are configured, this function + // picks the first listener in the list of listeners. Hence, users of this config must adjust their order accordingly. + String firstListener = listeners.split("\\s*,\\s*")[0]; + String[] protocolHostPort = firstListener.split(":"); + // Use port of listener only if no explicit config specified for KafkaConfig.PortProp(). + String portToUse = port == null ? protocolHostPort[protocolHostPort.length - 1] : String.valueOf(port); + // Use host of listener if one is specified. + return ((protocolHostPort[1].length() == 2) ? DEFAULT_BOOTSTRAP_SERVERS_HOST : protocolHostPort[1].substring(2)) + ":" + portToUse; + } + + return DEFAULT_BOOTSTRAP_SERVERS_HOST + ":" + (port == null ? DEFAULT_BOOTSTRAP_SERVERS_PORT : port); + } + + /** + * Starting with Kafka 3.3.0 a new class, "org.apache.kafka.server.metrics.KafkaYammerMetrics", provides the default Metrics Registry. + *

+ * This is the third default Metrics Registry class change since Kafka 2.5: + * - Metrics Registry class in Kafka 3.3+: org.apache.kafka.server.metrics.KafkaYammerMetrics + * - Metrics Registry class in Kafka 2.6+: kafka.metrics.KafkaYammerMetrics + * - Metrics Registry class in Kafka 2.5-: com.yammer.metrics.Metrics + *

+ * The older default registries do not work with the newer versions of Kafka. Therefore, if the new class exists, we use it and if + * it doesn't exist we will fall back on the older ones. + *

+ * Once CC supports only 2.6.0 and newer, we can clean this up and use only KafkaYammerMetrics all the time. + * + * @return MetricsRegistry with Kafka metrics + */ + private static MetricsRegistry metricsRegistry() { + Object metricsRegistry; + Class metricsClass; + + try { + // First we try to get the KafkaYammerMetrics class for Kafka 3.3+ + metricsClass = Class.forName(YAMMER_METRICS_IN_KAFKA_3_3_AND_LATER); + LOGGER.info("Found class {} for Kafka 3.3 and newer.", YAMMER_METRICS_IN_KAFKA_3_3_AND_LATER); + } catch (ClassNotFoundException e) { + LOGGER.info("Class {} not found. We are probably on Kafka 3.2 or older.", YAMMER_METRICS_IN_KAFKA_3_3_AND_LATER); + + // We did not find the KafkaYammerMetrics class from Kafka 3.3+. So we are probably on older Kafka version + // => we will try the older class for Kafka 2.6+. + try { + metricsClass = Class.forName(YAMMER_METRICS_IN_KAFKA_2_6_AND_LATER); + LOGGER.info("Found class {} for Kafka 2.6 and newer.", YAMMER_METRICS_IN_KAFKA_2_6_AND_LATER); + } catch (ClassNotFoundException ee) { + LOGGER.info("Class {} not found. We are probably on Kafka 2.5 or older.", YAMMER_METRICS_IN_KAFKA_2_6_AND_LATER); + + // We did not find the KafkaYammerMetrics class from Kafka 2.6+. So we are probably on older Kafka version + // => we will try the older class for Kafka 2.5-. + try { + metricsClass = Class.forName(YAMMER_METRICS_IN_KAFKA_2_5_AND_EARLIER); + LOGGER.info("Found class {} for Kafka 2.5 and earlier.", YAMMER_METRICS_IN_KAFKA_2_5_AND_EARLIER); + } catch (ClassNotFoundException eee) { + // No class was found for any Kafka version => we should fail + throw new RuntimeException("Failed to find Yammer Metrics class", eee); + } + } + } + + try { + Method method = metricsClass.getMethod("defaultRegistry"); + metricsRegistry = method.invoke(null); + } catch (InvocationTargetException | NoSuchMethodException | IllegalAccessException e) { + throw new RuntimeException("Failed to get metrics registry", e); + } + + if (metricsRegistry instanceof MetricsRegistry) { + return (MetricsRegistry) metricsRegistry; + } else { + throw new RuntimeException("Metrics registry does not have the expected type"); + } + } + + @Override + public void init(List metrics) { + metricsReporterRunner = new KafkaThread("AutoBalancerMetricsReporterRunner", this, true); + 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 + * + * @param name the name of the {@link Metric} + * @param metric the {@link Metric} + */ + @Override + public synchronized void onMetricAdded(MetricName name, Metric metric) { + addMetricIfInterested(name, metric); + } + + /** + * On yammer metric removed + * + * @param name the name of the {@link com.yammer.metrics.core.Metric} + */ + @Override + public synchronized void onMetricRemoved(MetricName name) { + interestedMetrics.remove(name); + } + + /** + * On kafka metric changed + * + * @param metric {@link KafkaMetric} + */ + @Override + public void metricChange(KafkaMetric metric) { + // do nothing, we only interested in yammer metrics now + } + + /** + * On kafka metric removed + * + * @param metric {@link KafkaMetric} + */ + @Override + public void metricRemoval(KafkaMetric metric) { + // do nothing, we only interested in yammer metrics now + } + + @Override + public void close() { + LOGGER.info("Closing Auto Balancer metrics reporter, id={}.", brokerId); + shutdown = true; + if (metricsReporterRunner != null) { + metricsReporterRunner.interrupt(); + } + if (producer != null) { + producer.close(PRODUCER_CLOSE_TIMEOUT); + } + } + + @Override + public void configure(Map configs) { + Properties producerProps = AutoBalancerMetricsReporterConfig.parseProducerConfigs(configs); + + //Add BootstrapServers if not set + if (!producerProps.containsKey(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)) { + String bootstrapServers = getBootstrapServers(configs); + producerProps.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + LOGGER.info("Using default value of {} for {}", bootstrapServers, + AutoBalancerMetricsReporterConfig.config(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + } + + //Add SecurityProtocol if not set + if (!producerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) { + String securityProtocol = "PLAINTEXT"; + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol); + LOGGER.info("Using default value of {} for {}", securityProtocol, + AutoBalancerMetricsReporterConfig.config(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); + } + + AutoBalancerMetricsReporterConfig reporterConfig = new AutoBalancerMetricsReporterConfig(configs, false); + + setIfAbsent(producerProps, + ProducerConfig.CLIENT_ID_CONFIG, + reporterConfig.getString(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID)); + setIfAbsent(producerProps, ProducerConfig.LINGER_MS_CONFIG, + reporterConfig.getLong(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_LINGER_MS_CONFIG).toString()); + setIfAbsent(producerProps, ProducerConfig.BATCH_SIZE_CONFIG, + reporterConfig.getInt(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_BATCH_SIZE_CONFIG).toString()); + setIfAbsent(producerProps, ProducerConfig.RETRIES_CONFIG, "5"); + setIfAbsent(producerProps, ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip"); + setIfAbsent(producerProps, ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + setIfAbsent(producerProps, ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, MetricSerde.class.getName()); + setIfAbsent(producerProps, ProducerConfig.ACKS_CONFIG, "all"); + + metricsReporterCreateRetries = reporterConfig.getInt( + AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_CREATE_RETRIES_CONFIG); + + createAutoBalancerMetricsProducer(producerProps); + if (producer == null) { + this.close(); + } + + brokerId = Integer.parseInt((String) configs.get(KafkaConfig.BrokerIdProp())); + brokerRack = (String) configs.get(KafkaConfig.RackProp()); + 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"); + } + + protected void createAutoBalancerMetricsProducer(Properties producerProps) throws KafkaException { + AutoBalancerMetricsUtils.retry(() -> { + try { + producer = new KafkaProducer<>(producerProps); + return false; + } catch (KafkaException e) { + if (e.getCause() instanceof ConfigException) { + // Check if the config exception is caused by bootstrap.servers config + try { + ProducerConfig config = new ProducerConfig(producerProps); + ClientUtils.parseAndValidateAddresses( + config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG), + config.getString(ProducerConfig.CLIENT_DNS_LOOKUP_CONFIG)); + } catch (ConfigException ce) { + // dns resolution may not be complete yet, let's retry again later + LOGGER.warn("Unable to create auto balancer metrics producer. ", ce.getCause()); + } + return true; + } + throw e; + } + }, metricsReporterCreateRetries); + } + + @Override + public void run() { + LOGGER.info("Starting auto balancer metrics reporter with reporting interval of {} ms.", reportingIntervalMs); + + try { + while (!shutdown) { + long now = System.currentTimeMillis(); + LOGGER.debug("Reporting metrics for time {}.", now); + try { + if (now > lastReportingTime + reportingIntervalMs) { + numMetricSendFailure = 0; + lastReportingTime = now; + reportMetrics(now); + } + try { + producer.flush(); + } catch (InterruptException ie) { + if (shutdown) { + LOGGER.info("auto balancer metric reporter is interrupted during flush due to shutdown request."); + } else { + throw ie; + } + } + } catch (Exception e) { + LOGGER.error("Got exception in auto balancer metrics reporter", e); + } + // Log failures if there is any. + if (numMetricSendFailure > 0) { + LOGGER.warn("Failed to send {} metrics for time {}", numMetricSendFailure, now); + } + numMetricSendFailure = 0; + long nextReportTime = now + reportingIntervalMs; + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Reporting finished for time {} in {} ms. Next reporting time {}", + now, System.currentTimeMillis() - now, nextReportTime); + } + while (!shutdown && now < nextReportTime) { + try { + Thread.sleep(nextReportTime - now); + } catch (InterruptedException ie) { + // let it go + } + now = System.currentTimeMillis(); + } + } + } finally { + LOGGER.info("auto balancer metrics reporter exited."); + } + } + + /** + * Send a AutoBalancerMetric to the Kafka topic. + * + * @param ccm the auto balancer metric to send. + */ + public void sendAutoBalancerMetric(AutoBalancerMetrics ccm) { + ProducerRecord producerRecord = + new ProducerRecord<>(autoBalancerMetricsTopic, null, ccm.time(), ccm.key(), ccm); + LOGGER.debug("Sending auto balancer metric {}.", ccm); + producer.send(producerRecord, (recordMetadata, e) -> { + if (e != null) { + numMetricSendFailure++; + } + }); + } + + private void reportMetrics(long now) throws Exception { + LOGGER.info("Reporting metrics."); + + YammerMetricProcessor.Context context = new YammerMetricProcessor.Context(now, brokerId, brokerRack, reportingIntervalMs); + processYammerMetrics(context); + processCpuMetrics(context); + for (Map.Entry entry : context.getMetricMap().entrySet()) { + sendAutoBalancerMetric(entry.getValue()); + } + + LOGGER.info("Finished reporting metrics, total metrics size: {}, merged size: {}.", interestedMetrics.size(), context.getMetricMap().size()); + } + + private void processYammerMetrics(YammerMetricProcessor.Context context) throws Exception { + for (Map.Entry entry : interestedMetrics.entrySet()) { + 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 + && !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); + } + } + } + + 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)) { + LOGGER.debug("Added new metric {} to auto balancer metrics reporter.", name); + interestedMetrics.put(name, metric); + } + } + + private void setIfAbsent(Properties props, String key, String value) { + if (!props.containsKey(key)) { + props.setProperty(key, value); + } + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsUtils.java b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsUtils.java new file mode 100644 index 0000000000..a14dc44fc5 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsUtils.java @@ -0,0 +1,221 @@ +/* + * 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. + */ +/* + * Some portion of this file 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; + +import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; + +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsUtils. + */ +public final class AutoBalancerMetricsUtils { + + public static final long ADMIN_CLIENT_CLOSE_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10); + public static final long CLIENT_REQUEST_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10); + private static final long DEFAULT_RETRY_BACKOFF_SCALE_MS = TimeUnit.SECONDS.toMillis(5); + private static final int DEFAULT_RETRY_BACKOFF_BASE = 2; + + private AutoBalancerMetricsUtils() { + + } + + private static void closeClientWithTimeout(Runnable clientCloseTask, long timeoutMs) { + Thread t = new Thread(clientCloseTask); + t.setDaemon(true); + t.start(); + try { + t.join(timeoutMs); + } catch (InterruptedException e) { + // let it go + } + if (t.isAlive()) { + t.interrupt(); + } + } + + /** + * Create an instance of AdminClient using the given configurations. + * + * @param adminClientConfigs Configurations used for the AdminClient. + * @return A new instance of AdminClient. + */ + public static AdminClient createAdminClient(Properties adminClientConfigs) { + return AdminClient.create(adminClientConfigs); + } + + /** + * Close the given AdminClient with the default timeout of {@link #ADMIN_CLIENT_CLOSE_TIMEOUT_MS}. + * + * @param adminClient AdminClient to be closed + */ + public static void closeAdminClientWithTimeout(AdminClient adminClient) { + closeAdminClientWithTimeout(adminClient, ADMIN_CLIENT_CLOSE_TIMEOUT_MS); + } + + /** + * Close the given AdminClient with the given timeout. + * + * @param adminClient AdminClient to be closed. + * @param timeoutMs the timeout. + */ + public static void closeAdminClientWithTimeout(AdminClient adminClient, long timeoutMs) { + closeClientWithTimeout(() -> { + try { + ((AutoCloseable) adminClient).close(); + } catch (Exception e) { + throw new IllegalStateException("Failed to close the Admin Client.", e); + } + }, timeoutMs); + } + + /** + * Parse AdminClient configs based on the given {@link AutoBalancerMetricsReporterConfig configs}. + * + * @param adminClientConfigs Configs that will be return with SSL configs. + * @param configs Configs to be used for parsing AdminClient SSL configs. + * @return AdminClient configs. + */ + public static Properties addSslConfigs(Properties adminClientConfigs, AutoBalancerMetricsReporterConfig configs) { + // Add security protocol (if specified). + try { + String securityProtocol = configs.getString(AdminClientConfig.SECURITY_PROTOCOL_CONFIG); + adminClientConfigs.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); + setStringConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_MECHANISM); + setPasswordConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_JAAS_CONFIG); + + // Configure SSL configs (if security protocol is SSL or SASL_SSL) + if (securityProtocol.equals(SecurityProtocol.SSL.name) || securityProtocol.equals(SecurityProtocol.SASL_SSL.name)) { + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_TYPE_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); + setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG); + setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEY_PASSWORD_CONFIG); + setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG); + } + } catch (ConfigException ce) { + // let it go. + } + + return adminClientConfigs; + } + + private static void setPasswordConfigIfExists(AutoBalancerMetricsReporterConfig configs, Properties props, String name) { + try { + props.put(name, configs.getPassword(name)); + } catch (ConfigException ce) { + // let it go. + } + } + + private static void setStringConfigIfExists(AutoBalancerMetricsReporterConfig configs, Properties props, String name) { + try { + props.put(name, configs.getString(name)); + } catch (ConfigException ce) { + // let it go. + } + } + + /** + * Create a config altering operation if config's current value does not equal to target value. + * + * @param configsToAlter Set of config altering operations to be applied. + * @param configsToSet Configs to set. + * @param currentConfig Current value of the config. + */ + public static void maybeUpdateConfig(Set configsToAlter, + Map configsToSet, + Config currentConfig) { + for (Map.Entry entry : configsToSet.entrySet()) { + String configName = entry.getKey(); + String targetConfigValue = entry.getValue(); + if (currentConfig.get(configName) == null || !currentConfig.get(configName).value().equals(targetConfigValue)) { + configsToAlter.add(new AlterConfigOp(new ConfigEntry(configName, targetConfigValue), AlterConfigOp.OpType.SET)); + } + } + } + + /** + * Retries the {@code Supplier} function while it returns {@code true} and for the specified max number of attempts. + * The delay between each attempt is computed as: delay = scaleMs * base ^ attempt + * + * @param function the code to call and retry if needed + * @param scaleMs the scale for computing the delay + * @param base the base for computing the delay + * @param maxAttempts the max number of attempts on calling the function + * @return {@code false} if the function requires a retry, but it cannot be retried, because the max attempts have been exceeded. + * {@code true} if the function stopped requiring a retry before exceeding the max attempts. + */ + public static boolean retry(Supplier function, long scaleMs, int base, int maxAttempts) { + if (maxAttempts > 0) { + int attempts = 0; + long timeToSleep = scaleMs; + boolean retry; + do { + retry = function.get(); + if (retry) { + try { + if (++attempts == maxAttempts) { + return false; + } + timeToSleep *= base; + Thread.sleep(timeToSleep); + } catch (InterruptedException ignored) { + + } + } + } while (retry); + } else { + throw new ConfigException("Max attempts has to be greater than zero."); + } + return true; + } + + /** + * Retries the {@code Supplier} function while it returns {@code true} and for the specified max number of attempts. + * It uses {@code DEFAULT_RETRY_BACKOFF_SCALE_MS} and {@code DEFAULT_RETRY_BACKOFF_BASE} for scale and base to compute the delay. + * + * @param function the code to call and retry if needed + * @param maxAttempts the max number of attempts on calling the function + * @return {@code false} if the function requires a retry, but it cannot be retried, because the max attempts have been exceeded. + * {@code true} if the function stopped requiring a retry before exceeding the max attempts. + */ + public static boolean retry(Supplier function, int maxAttempts) { + return retry(function, DEFAULT_RETRY_BACKOFF_SCALE_MS, DEFAULT_RETRY_BACKOFF_BASE, maxAttempts); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/exception/AutoBalancerMetricsReporterException.java b/core/src/main/java/kafka/autobalancer/metricsreporter/exception/AutoBalancerMetricsReporterException.java new file mode 100644 index 0000000000..93a0709e1e --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/exception/AutoBalancerMetricsReporterException.java @@ -0,0 +1,39 @@ +/* + * 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.exception; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.exception.CruiseControlMetricsReporterException. + */ +public class AutoBalancerMetricsReporterException extends Exception { + + public AutoBalancerMetricsReporterException(String message, Throwable cause) { + super(message, cause); + } + + public AutoBalancerMetricsReporterException(String message) { + super(message); + } + + public AutoBalancerMetricsReporterException(Throwable cause) { + super(cause); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/exception/UnknownVersionException.java b/core/src/main/java/kafka/autobalancer/metricsreporter/exception/UnknownVersionException.java new file mode 100644 index 0000000000..43f437cbe4 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/exception/UnknownVersionException.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. + */ +/* + * 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.exception; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.exception.UnknownVersionException. + */ +/* + Unknown version during Serialization/Deserialization. + */ +public class UnknownVersionException extends AutoBalancerMetricsReporterException { + public UnknownVersionException(String msg) { + super(msg); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java new file mode 100644 index 0000000000..2036c3f135 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/AutoBalancerMetrics.java @@ -0,0 +1,165 @@ +/* + * 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. + */ +/* + * Some portion of this file 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.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.CruiseControlMetric. + */ +/* + * 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<>(); + private final long time; + private final int brokerId; + private final String brokerRack; + + public AutoBalancerMetrics(long time, int brokerId, String brokerRack) { + this(time, brokerId, brokerRack, Collections.emptyMap()); + } + + public AutoBalancerMetrics(long time, int brokerId, String brokerRack, Map metricTypeValueMap) { + this.time = time; + this.brokerId = brokerId; + this.brokerRack = brokerRack; + this.metricTypeValueMap.putAll(metricTypeValueMap); + } + + static Map parseMetricsMap(ByteBuffer buffer) { + Map metricsMap = new HashMap<>(); + int metricNumber = buffer.getInt(); + for (int i = 0; i < metricNumber; i++) { + byte id = buffer.get(); + double value = buffer.getDouble(); + metricsMap.put(RawMetricType.forId(id), value); + } + return metricsMap; + } + + public AutoBalancerMetrics put(RawMetricType type, double value) { + this.metricTypeValueMap.put(type, value); + return this; + } + + public void add(AutoBalancerMetrics metrics) { + for (Map.Entry metricEntry : metrics.metricTypeValueMap.entrySet()) { + this.metricTypeValueMap.putIfAbsent(metricEntry.getKey(), metricEntry.getValue()); + } + } + + public Map getMetricTypeValueMap() { + return metricTypeValueMap; + } + + public abstract String key(); + + /** + * @return the metric class id for this metric. The metric class id will be stored in the serialized metrics + * so that the deserializer will know which class should be used to deserialize the data. + */ + public abstract MetricClassId metricClassId(); + + /** + * @return the timestamp for this metric. + */ + public long time() { + return time; + } + + /** + * @return the broker id who reported this metric. + */ + public int brokerId() { + return brokerId; + } + + public String brokerRack() { + return brokerRack; + } + + public int bodySize() { + return Integer.SIZE + (Byte.SIZE + Double.SIZE) * metricTypeValueMap.size(); + } + + public ByteBuffer writeBody(ByteBuffer buffer) { + buffer.putInt(metricTypeValueMap.size()); + for (Map.Entry entry : metricTypeValueMap.entrySet()) { + buffer.put(entry.getKey().id()); + buffer.putDouble(entry.getValue()); + } + return buffer; + } + + /** + * Serialize the metric to a byte buffer with the header size reserved. + * + * @param headerSize the header size to reserve. + * @return A ByteBuffer with header size reserved at the beginning. + */ + abstract ByteBuffer toBuffer(int headerSize); + + public String buildKVString() { + StringBuilder builder = new StringBuilder(); + for (Map.Entry entry : metricTypeValueMap.entrySet()) { + builder.append(entry.getKey()); + builder.append(":"); + builder.append(String.format("%.4f", entry.getValue())); + } + return builder.toString(); + } + + @Override + 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/BrokerMetrics.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/BrokerMetrics.java new file mode 100644 index 0000000000..d07e0ac5ae --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/BrokerMetrics.java @@ -0,0 +1,124 @@ +/* + * 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. + */ +/* + * Some portion of this file 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 kafka.autobalancer.metricsreporter.exception.UnknownVersionException; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.BrokerMetrics. + */ +/* + * A container class to hold broker metric. + */ +public class BrokerMetrics extends AutoBalancerMetrics { + + public BrokerMetrics(long time, int brokerId, String brokerRack) { + super(time, brokerId, brokerRack); + } + + public BrokerMetrics(long time, int brokerId, String brokerRack, Map metricTypeValueMap) { + super(time, brokerId, brokerRack, metricTypeValueMap); + } + + static BrokerMetrics fromBuffer(ByteBuffer buffer) throws UnknownVersionException { + byte version = buffer.get(); + if (version > METRIC_VERSION) { + throw new UnknownVersionException("Cannot deserialize the topic metrics for version " + version + ". " + + "Current version is " + METRIC_VERSION); + } + long time = buffer.getLong(); + int brokerId = buffer.getInt(); + int brokerRackLength = buffer.getInt(); + String brokerRack = ""; + if (brokerRackLength > 0) { + brokerRack = new String(buffer.array(), buffer.arrayOffset() + buffer.position(), brokerRackLength, StandardCharsets.UTF_8); + buffer.position(buffer.position() + brokerRackLength); + } + Map metricsMap = parseMetricsMap(buffer); + return new BrokerMetrics(time, brokerId, brokerRack, metricsMap); + } + + @Override + public AutoBalancerMetrics put(RawMetricType type, double value) { + if (type.metricScope() != RawMetricType.MetricScope.BROKER) { + throw new IllegalArgumentException(String.format("Cannot construct a BrokerMetric for %s whose scope is %s", + type, type.metricScope())); + } + return super.put(type, value); + } + + @Override + public String key() { + return Integer.toString(brokerId()); + } + + @Override + public MetricClassId metricClassId() { + return MetricClassId.BROKER_METRIC; + } + + /** + * The buffer capacity is calculated as follows: + *

    + *
  • (headerPos + {@link Byte#BYTES}) - version
  • + *
  • {@link Long#BYTES} - time
  • + *
  • {@link Integer#BYTES} - broker id
  • + *
  • {@link Integer#BYTES} - broker rack length
  • + *
  • brokerRack.length - broker rack
  • + *
  • body length - metric-value body
  • + * + *
+ * + * @param headerPos Header position + * @return Byte buffer of the partition metric. + */ + @Override + public ByteBuffer toBuffer(int headerPos) { + byte[] brokerRackBytes = brokerRack().getBytes(StandardCharsets.UTF_8); + ByteBuffer buffer = ByteBuffer.allocate(headerPos + Byte.BYTES + + Long.BYTES + + Integer.BYTES + + Integer.BYTES + + brokerRackBytes.length + + bodySize()); + buffer.position(headerPos); + buffer.put(METRIC_VERSION); + buffer.putLong(time()); + buffer.putInt(brokerId()); + buffer.putInt(brokerRackBytes.length); + if (brokerRackBytes.length > 0) { + buffer.put(brokerRackBytes); + } + buffer = writeBody(buffer); + return buffer; + } + + @Override + public String toString() { + return String.format("[%s,BrokerId=%d,Time=%d,Key:Value=%s]", + MetricClassId.BROKER_METRIC, brokerId(), time(), buildKVString()); + } +} + diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java new file mode 100644 index 0000000000..f89bb7b905 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/ContainerMetricUtils.java @@ -0,0 +1,144 @@ +/* + * 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/EmptyMeter.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/EmptyMeter.java new file mode 100644 index 0000000000..777c01e2b3 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/EmptyMeter.java @@ -0,0 +1,66 @@ +/* + * 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; + +import com.yammer.metrics.core.Metered; +import com.yammer.metrics.core.MetricName; +import com.yammer.metrics.core.MetricProcessor; + +import java.util.concurrent.TimeUnit; + +public class EmptyMeter implements Metered { + @Override + public TimeUnit rateUnit() { + return null; + } + + @Override + public String eventType() { + return null; + } + + @Override + public long count() { + return 0; + } + + @Override + public double fifteenMinuteRate() { + return 0; + } + + @Override + public double fiveMinuteRate() { + return 0; + } + + @Override + public double meanRate() { + return 0; + } + + @Override + public double oneMinuteRate() { + return 0; + } + + @Override + public void processWith(MetricProcessor processor, MetricName name, T context) throws Exception { + processor.processMeter(name, this, context); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java new file mode 100644 index 0000000000..fc077608ad --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricSerde.java @@ -0,0 +1,94 @@ +/* + * 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. + */ +/* + * Some portion of this file 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 kafka.autobalancer.metricsreporter.exception.UnknownVersionException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.nio.ByteBuffer; +import java.util.Map; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.MetricSerde. + */ +public class MetricSerde implements Serializer, Deserializer { + + // The overhead of the type bytes + private static final int METRIC_TYPE_OFFSET = 0; + private static final int HEADER_LENGTH = 1; + + /** + * Serialize the Auto Balancer metric to a byte array. + * + * @param metric Metric to be serialized. + * @return Serialized Auto Balancer metric as a byte array. + */ + public static byte[] toBytes(AutoBalancerMetrics metric) { + ByteBuffer byteBuffer = metric.toBuffer(HEADER_LENGTH); + byteBuffer.put(METRIC_TYPE_OFFSET, metric.metricClassId().id()); + return byteBuffer.array(); + } + + /** + * Deserialize from byte array to Auto Balancer metric + * + * @param bytes Bytes array corresponding to Auto Balancer metric. + * @return Deserialized byte array as Auto Balancer metric. + */ + public static AutoBalancerMetrics fromBytes(byte[] bytes) throws UnknownVersionException { + ByteBuffer buffer = ByteBuffer.wrap(bytes); + switch (AutoBalancerMetrics.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. + // simply ignore the metric by returning a null. + return null; + } + } + + @Override + public AutoBalancerMetrics deserialize(String topic, byte[] bytes) { + try { + return fromBytes(bytes); + } catch (Exception e) { + throw new RuntimeException("Error occurred when deserialize auto balancer metrics.", e); + } + } + + @Override + public void configure(Map map, boolean b) { + + } + + @Override + public byte[] serialize(String s, AutoBalancerMetrics autoBalancerMetric) { + return toBytes(autoBalancerMetric); + } + + @Override + public void close() { + + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java new file mode 100644 index 0000000000..e03abbb711 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/MetricsUtils.java @@ -0,0 +1,327 @@ +/* + * 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. + */ +/* + * Some portion of this file 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 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; +import java.util.Map; +import java.util.Set; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.MetricsUtils. + */ +public final class MetricsUtils { + // Names + public static final String BYTES_IN_PER_SEC = "BytesInPerSec"; + public static final String BYTES_OUT_PER_SEC = "BytesOutPerSec"; + /* Used to identify idle partitions */ + public static final String SIZE = "Size"; + // Attribute + static final String ATTRIBUTE_MEAN = "Mean"; + static final String ATTRIBUTE_MAX = "Max"; + static final String ATTRIBUTE_50TH_PERCENTILE = "50thPercentile"; + static final String ATTRIBUTE_999TH_PERCENTILE = "999thPercentile"; + // Groups + private static final String KAFKA_SERVER = "kafka.server"; + private static final String KAFKA_LOG_PREFIX = "kafka.log"; + // Type Keys + private static final String TOPIC_KEY = "topic"; + private static final String PARTITION_KEY = "partition"; + // 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); + private static final Set INTERESTED_LOG_METRIC_NAMES = Set.of(SIZE); + + private MetricsUtils() { + + } + + /** + * Get empty metric for specified metric name + * + * @param name metric name + * @return empty metric implementation + */ + public static Metric getEmptyMetricFor(String name) { + switch (name) { + case BYTES_IN_PER_SEC: + case BYTES_OUT_PER_SEC: + return new EmptyMeter(); + default: + return null; + } + } + + 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; + String type = null; + if (BYTES_IN_PER_SEC.equals(name) || BYTES_OUT_PER_SEC.equals(name)) { + group = KAFKA_SERVER; + type = BROKER_TOPIC_PARTITION_METRICS_GROUP; + } + if (group == null) { + return null; + } + return KafkaMetricsGroup$.MODULE$.explicitMetricName(group, type, name, CollectionConverters.asScala(tags)); + } + + public static Set getMetricNameMaybeMissing() { + return new HashSet<>(INTERESTED_TOPIC_PARTITION_METRIC_NAMES); + } + + /** + * Create a Auto Balancer Metric. + * + * @param nowMs The current time in milliseconds. + * @param brokerId Broker Id. + * @param metricName Yammer metric name. + * @param value Metric value + * @return A Yammer metric converted as a AutoBalancerMetric. + */ + public static AutoBalancerMetrics toAutoBalancerMetric(long nowMs, + int brokerId, + String brokerRack, + com.yammer.metrics.core.MetricName metricName, + double value) { + return toAutoBalancerMetric(nowMs, brokerId, brokerRack, metricName, value, null); + } + + /** + * Create a Auto Balancer Metric. + * + * @param nowMs The current time in milliseconds. + * @param brokerId Broker Id. + * @param brokerRack Broker rack. + * @param metricName Yammer metric name. + * @param value Metric value + * @param attribute Metric attribute. + * @return A Yammer metric converted as a AutoBalancerMetric. + */ + public static AutoBalancerMetrics toAutoBalancerMetric(long nowMs, + int brokerId, + String brokerRack, + com.yammer.metrics.core.MetricName metricName, + double value, + String attribute) { + Map tags = yammerMetricScopeToTags(metricName.getScope()); + AutoBalancerMetrics ccm = tags == null ? null : toAutoBalancerMetric(nowMs, brokerId, brokerRack, metricName.getName(), tags, value, attribute); + if (ccm == null) { + throw new IllegalArgumentException(String.format("Cannot convert yammer metric %s to a Auto Balancer metric for " + + "broker %d at time %d for tag %s", metricName, brokerId, nowMs, attribute)); + } + return ccm; + } + + /** + * Build a AutoBalancerMetric object. + * + * @param nowMs The current time in milliseconds. + * @param brokerId Broker Id. + * @param brokerRack Broker rack. + * @param name Name of the metric. + * @param tags Tags of the metric. + * @param value Metric value. + * @param attribute Metric attribute -- can be {@code null}. + * @return A {@link AutoBalancerMetrics} object with the given properties. + */ + private static AutoBalancerMetrics toAutoBalancerMetric(long nowMs, + int brokerId, + String brokerRack, + String name, + Map tags, + double value, + String attribute) { + String topic = tags.get(TOPIC_KEY); + String partitionStr = tags.get(PARTITION_KEY); + int partition = -1; + if (partitionStr != null) { + try { + partition = Integer.parseInt(partitionStr); + } catch (NumberFormatException e) { + return null; + } + } + switch (name) { + case BYTES_IN_PER_SEC: + // network inbound bandwidth capacity is in KB/s + value = value / 1024; + return bytesInToMetric(topic, partition, nowMs, brokerId, brokerRack, value); + case BYTES_OUT_PER_SEC: + // network inbound bandwidth capacity is in KB/s + value = value / 1024; + return bytesOutToMetric(topic, partition, nowMs, brokerId, brokerRack, value); + case SIZE: + if (partition == -1) { + return null; + } + return new TopicPartitionMetrics(nowMs, brokerId, brokerRack, topic, partition).put(RawMetricType.PARTITION_SIZE, value); + default: + return null; + } + } + + /** + * 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. + * + * @param metricName Yammer metric name. + * @return {@code true} if the yammer metric name is an interested metric, {@code false} otherwise. + */ + public static boolean isInterested(com.yammer.metrics.core.MetricName metricName) { + Map tags = yammerMetricScopeToTags(metricName.getScope()); + return tags != null && isInterested(metricName.getGroup(), metricName.getName(), metricName.getType(), tags); + } + + /** + * Check if a metric is an interested metric. + * + * @param group Group of the metric. + * @param name Name of the metric. + * @param type Type of the metric. + * @param tags Tags of the metric. + * @return {@code true} for a metric of interest, {@code false} otherwise. + */ + private static boolean isInterested(String group, String name, String type, Map tags) { + 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); + } + return false; + } + + /** + * Convert a yammer metrics scope to a tags map. + * + * @param scope Scope of the Yammer metric. + * @return Empty map for {@code null} scope, {@code null} for scope with keys without a matching value (i.e. unacceptable + * scope) (see ...), parsed tags otherwise. + */ + public static Map yammerMetricScopeToTags(String scope) { + if (scope != null) { + String[] kv = scope.split("\\."); + if (kv.length % 2 != 0) { + return null; + } + Map tags = new HashMap<>(); + for (int i = 0; i < kv.length; i += 2) { + tags.put(kv[i], kv[i + 1]); + } + return tags; + } else { + return Collections.emptyMap(); + } + } + + /** + * Check if tags are valid for a topic-partition metric + * + * @param tags metric tags + * @return true if valid, false otherwise + */ + public static boolean sanityCheckTopicPartitionTags(Map tags) { + return tags.containsKey("topic") && tags.containsKey("partition"); + } + + private static AutoBalancerMetrics bytesInToMetric(String topic, int partition, long nowMs, + int brokerId, String brokerRack, double value) { + + 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; + } + + private static AutoBalancerMetrics bytesOutToMetric(String topic, int partition, long nowMs, + int brokerId, String brokerRack, double value) { + + 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()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/RawMetricType.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/RawMetricType.java new file mode 100644 index 0000000000..48c0d39beb --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/RawMetricType.java @@ -0,0 +1,139 @@ +/* + * 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. + */ +/* + * Some portion of this file 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.util.ArrayList; +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; + +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.MetricScope.BROKER; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.MetricScope.PARTITION; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.RawMetricType. + */ +/* + * The metric type helps the metric sampler to distinguish what metric a value is representing. These metrics are + * called raw metrics because they are the most basic information reported by the Kafka brokers without any processing. + * Each metric type has an id for serde purpose. + */ +public enum RawMetricType { + BROKER_CAPACITY_NW_IN(BROKER, (byte) 0, (byte) 0), + BROKER_CAPACITY_NW_OUT(BROKER, (byte) 1, (byte) 0), + ALL_TOPIC_BYTES_IN(BROKER, (byte) 2, (byte) 0), + ALL_TOPIC_BYTES_OUT(BROKER, (byte) 3, (byte) 0), + TOPIC_PARTITION_BYTES_IN(PARTITION, (byte) 4, (byte) 0), + TOPIC_PARTITION_BYTES_OUT(PARTITION, (byte) 5, (byte) 0), + PARTITION_SIZE(PARTITION, (byte) 6, (byte) 0), + BROKER_CPU_UTIL(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(BROKER)); + private static final List PARTITION_METRIC_TYPES = Collections.unmodifiableList(buildMetricTypeList(PARTITION)); + private final byte id; + private final MetricScope metricScope; + private final byte supportedVersionSince; + + RawMetricType(MetricScope scope, byte id) { + this(scope, id, (byte) -1); + } + + RawMetricType(MetricScope scope, byte id, byte supportedVersionSince) { + this.id = id; + metricScope = scope; + 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; + } + + public static List brokerMetricTypes() { + return BROKER_METRIC_TYPES; + } + + /** + * @param id Auto Balancer Metric type. + * @return Raw metric type. + */ + public static RawMetricType forId(byte id) { + if (id < values().length) { + return values()[id]; + } else { + throw new IllegalArgumentException("AutoBalancerMetric type " + id + " does not exist."); + } + } + + private static SortedMap> buildBrokerMetricTypesDiffByVersion() { + SortedMap> buildBrokerMetricTypesDiffByVersion = new TreeMap<>(); + for (RawMetricType type : RawMetricType.values()) { + if (type.metricScope() == BROKER) { + buildBrokerMetricTypesDiffByVersion.computeIfAbsent(type.supportedVersionSince(), t -> new HashSet<>()).add(type); + } + } + + return buildBrokerMetricTypesDiffByVersion; + } + + private static List buildMetricTypeList(MetricScope metricScope) { + List brokerMetricTypes = new ArrayList<>(); + for (RawMetricType type : RawMetricType.values()) { + if (type.metricScope() == metricScope) { + brokerMetricTypes.add(type); + } + } + return brokerMetricTypes; + } + + public byte id() { + return id; + } + + public MetricScope metricScope() { + return metricScope; + } + + public byte supportedVersionSince() { + return supportedVersionSince; + } + + public enum MetricScope { + BROKER, PARTITION + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java new file mode 100644 index 0000000000..9945696003 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/TopicPartitionMetrics.java @@ -0,0 +1,149 @@ +/* + * 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. + */ +/* + * Some portion of this file 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 kafka.autobalancer.metricsreporter.exception.UnknownVersionException; +import org.apache.kafka.common.TopicPartition; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.Map; + +/** + * 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; + + public TopicPartitionMetrics(long time, int brokerId, String brokerRack, String topic, int partition) { + this(time, brokerId, brokerRack, topic, partition, Collections.emptyMap()); + } + + public TopicPartitionMetrics(long time, int brokerId, String brokerRack, String topic, int partition, Map metricsMap) { + super(time, brokerId, brokerRack, metricsMap); + this.topic = topic; + this.partition = partition; + } + + static TopicPartitionMetrics fromBuffer(ByteBuffer buffer) throws UnknownVersionException { + byte version = buffer.get(); + if (version > METRIC_VERSION) { + throw new UnknownVersionException("Cannot deserialize the topic metrics for version " + version + ". " + + "Current version is " + METRIC_VERSION); + } + long time = buffer.getLong(); + int brokerId = buffer.getInt(); + + int brokerRackLength = buffer.getInt(); + String brokerRack = ""; + if (brokerRackLength > 0) { + brokerRack = new String(buffer.array(), buffer.arrayOffset() + buffer.position(), brokerRackLength, StandardCharsets.UTF_8); + buffer.position(buffer.position() + brokerRackLength); + } + int topicLength = buffer.getInt(); + String topic = new String(buffer.array(), buffer.arrayOffset() + buffer.position(), topicLength, StandardCharsets.UTF_8); + buffer.position(buffer.position() + topicLength); + int partition = buffer.getInt(); + Map metricsMap = parseMetricsMap(buffer); + return new TopicPartitionMetrics(time, brokerId, brokerRack, topic, partition, metricsMap); + } + + @Override + public AutoBalancerMetrics put(RawMetricType type, double value) { + if (type.metricScope() != RawMetricType.MetricScope.PARTITION) { + throw new IllegalArgumentException(String.format("Cannot construct a PartitionMetric for %s whose scope is %s", + type, type.metricScope())); + } + return super.put(type, value); + } + + @Override + public String key() { + return topic + "-" + partition; + } + + public MetricClassId metricClassId() { + return MetricClassId.PARTITION_METRIC; + } + + public String topic() { + return topic; + } + + public int partition() { + return partition; + } + + /** + * The buffer capacity is calculated as follows: + *

    + *
  • (headerPos + {@link Byte#BYTES}) - version
  • + *
  • {@link Long#BYTES} - time
  • + *
  • {@link Integer#BYTES} - broker id
  • + *
  • {@link Integer#BYTES} - broker rack length
  • + *
  • brokerRack.length - broker rack
  • + *
  • {@link Integer#BYTES} - topic length
  • + *
  • topic.length - topic
  • + *
  • {@link Integer#BYTES} - partition
  • + *
  • body length - metric-value body
  • + *
+ * + * @param headerPos Header position + * @return Byte buffer of the partition metric. + */ + public ByteBuffer toBuffer(int headerPos) { + byte[] brokerRackBytes = brokerRack().getBytes(StandardCharsets.UTF_8); + byte[] topic = topic().getBytes(StandardCharsets.UTF_8); + ByteBuffer buffer = ByteBuffer.allocate(headerPos + Byte.BYTES + + Long.BYTES + + Integer.BYTES + + Integer.BYTES + + brokerRackBytes.length + + Integer.BYTES + + topic.length + + Integer.BYTES + + bodySize()); + buffer.position(headerPos); + buffer.put(METRIC_VERSION); + buffer.putLong(time()); + buffer.putInt(brokerId()); + buffer.putInt(brokerRackBytes.length); + if (brokerRackBytes.length > 0) { + buffer.put(brokerRackBytes); + } + buffer.putInt(topic.length); + buffer.put(topic); + buffer.putInt(partition); + + buffer = writeBody(buffer); + return buffer; + } + + @Override + public String toString() { + return String.format("[%s,Time=%d,BrokerId=%d,Partition=%s,Key:Value=%s]", + MetricClassId.PARTITION_METRIC, time(), brokerId(), + new TopicPartition(topic(), partition()), buildKVString()); + } +} diff --git a/core/src/main/java/kafka/autobalancer/metricsreporter/metric/YammerMetricProcessor.java b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/YammerMetricProcessor.java new file mode 100644 index 0000000000..bebb68898f --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/metricsreporter/metric/YammerMetricProcessor.java @@ -0,0 +1,230 @@ +/* + * 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. + */ +/* + * Some portion of this file 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 com.yammer.metrics.core.Counter; +import com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Histogram; +import com.yammer.metrics.core.Metered; +import com.yammer.metrics.core.MetricName; +import com.yammer.metrics.core.MetricProcessor; +import com.yammer.metrics.core.Timer; +import com.yammer.metrics.stats.Snapshot; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * This class was modified based on Cruise Control: com.linkedin.kafka.cruisecontrol.metricsreporter.metric.YammerMetricProcessor. + */ +/* + * A Yammer metric processor that process the yammer metrics. Currently all the interested metrics are of type + * Meter (BytesInRate, BytesOutRate) or Gauge (Partition Size). + */ +public class YammerMetricProcessor implements MetricProcessor { + private static final Logger LOG = LoggerFactory.getLogger(YammerMetricProcessor.class); + + @Override + public void processMeter(MetricName metricName, Metered metered, Context context) { + if (MetricsUtils.isInterested(metricName)) { + LOG.trace("Processing metric {} of type Meter.", metricName); + double value; + if (context.reportingInterval().toMillis() <= TimeUnit.MINUTES.toMillis(1)) { + value = metered.oneMinuteRate(); + } else if (context.reportingInterval().toMillis() <= TimeUnit.MINUTES.toMillis(5)) { + value = metered.fiveMinuteRate(); + } else { + value = metered.fifteenMinuteRate(); + } + AutoBalancerMetrics ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + value); + context.merge(ccm); + } + } + + @Override + public void processCounter(MetricName metricName, Counter counter, Context context) { + if (MetricsUtils.isInterested(metricName)) { + LOG.warn("Not processing metric {} of type Counter.", metricName); + } + } + + @Override + public void processHistogram(MetricName metricName, Histogram histogram, Context context) { + if (MetricsUtils.isInterested(metricName)) { + LOG.trace("Processing metric {} of type Histogram.", metricName); + // Get max metric value + AutoBalancerMetrics ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + histogram.max(), + MetricsUtils.ATTRIBUTE_MAX); + context.merge(ccm); + + // Get mean metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + histogram.mean(), + MetricsUtils.ATTRIBUTE_MEAN); + context.merge(ccm); + + Snapshot snapshot = histogram.getSnapshot(); + // Get 50th percentile (i.e. median) metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + snapshot.getMedian(), + MetricsUtils.ATTRIBUTE_50TH_PERCENTILE); + context.merge(ccm); + + // Get 999th percentile metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + snapshot.get999thPercentile(), + MetricsUtils.ATTRIBUTE_999TH_PERCENTILE); + context.merge(ccm); + } + } + + @Override + public void processTimer(MetricName metricName, Timer timer, Context context) { + if (MetricsUtils.isInterested(metricName)) { + LOG.trace("Processing metric {} of type Timer.", metricName); + + AutoBalancerMetrics ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + timer.fiveMinuteRate()); + context.merge(ccm); + // Get max metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + timer.max(), + MetricsUtils.ATTRIBUTE_MAX); + context.merge(ccm); + // Get mean metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + timer.mean(), + MetricsUtils.ATTRIBUTE_MEAN); + context.merge(ccm); + + Snapshot snapshot = timer.getSnapshot(); + // Get 50th percentile (i.e. median) metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + snapshot.getMedian(), + MetricsUtils.ATTRIBUTE_50TH_PERCENTILE); + context.merge(ccm); + + // Get 999th percentile metric value + ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + snapshot.get999thPercentile(), + MetricsUtils.ATTRIBUTE_999TH_PERCENTILE); + context.merge(ccm); + } + } + + @Override + public void processGauge(MetricName metricName, Gauge gauge, Context context) { + if (MetricsUtils.isInterested(metricName)) { + LOG.trace("Processing metric {} of type Gauge.", metricName); + if (!(gauge.value() instanceof Number)) { + throw new IllegalStateException(String.format("The value of yammer metric %s is %s, which is not a number", + metricName, gauge.value())); + } + AutoBalancerMetrics ccm = MetricsUtils.toAutoBalancerMetric(context.time(), + context.brokerId(), + context.brokerRack(), + metricName, + ((Number) gauge.value()).doubleValue()); + context.merge(ccm); + } + } + + public static final class Context { + private final long time; + private final int brokerId; + private final String brokerRack; + private final Duration reportingInterval; + + private final Map metricMap; + + public Context(long time, int brokerId, String brokerRack, long reportingIntervalMs) { + this.time = time; + this.brokerId = brokerId; + this.brokerRack = brokerRack; + this.reportingInterval = Duration.ofMillis(reportingIntervalMs); + this.metricMap = new HashMap<>(); + } + + public void merge(AutoBalancerMetrics metric) { + String mergeKey = metric.key(); + AutoBalancerMetrics prev = metricMap.putIfAbsent(mergeKey, metric); + if (prev != null) { + prev.add(metric); + } + } + + public Map getMetricMap() { + return metricMap; + } + + public long time() { + return time; + } + + public int brokerId() { + return brokerId; + } + + private String brokerRack() { + return brokerRack; + } + + private Duration reportingInterval() { + return reportingInterval; + } + } +} diff --git a/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java b/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java new file mode 100644 index 0000000000..14007438cf --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/BrokerUpdater.java @@ -0,0 +1,239 @@ +/* + * 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 kafka.autobalancer.common.Resource; +import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics; +import kafka.autobalancer.metricsreporter.metric.MetricsUtils; +import kafka.autobalancer.metricsreporter.metric.RawMetricType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +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(); + private final Broker broker; + + public BrokerUpdater(int brokerId) { + this.broker = new Broker(brokerId); + } + + public static class Broker { + 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 boolean active; + private long timestamp; + + public Broker(int brokerId) { + this.brokerId = brokerId; + } + + public Broker(Broker 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 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 == 0) { + return 0; + } + return load(resource) / capacity(resource); + } + + public void setActive(boolean active) { + this.active = active; + } + + public boolean isActive() { + return this.active; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public long getTimestamp() { + return this.timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Broker broker = (Broker) o; + return brokerId == broker.brokerId; + } + + @Override + public int hashCode() { + return Objects.hashCode(brokerId); + } + + @Override + public String toString() { + return "Broker{" + + "brokerId=" + brokerId + + ", brokerCapacity=" + Arrays.toString(brokerCapacity) + + ", brokerLoad=" + Arrays.toString(brokerLoad) + + ", active=" + active + + '}'; + } + } + + public boolean update(AutoBalancerMetrics metrics) { + if (metrics.metricClassId() != AutoBalancerMetrics.MetricClassId.BROKER_METRIC) { + LOGGER.error("Mismatched metrics type {} for broker", metrics.metricClassId()); + return false; + } + + if (!MetricsUtils.sanityCheckBrokerMetricsCompleteness(metrics)) { + LOGGER.error("Broker metrics sanity check failed, metrics is incomplete {}", metrics); + return false; + } + + lock.lock(); + try { + if (metrics.time() < broker.getTimestamp()) { + LOGGER.warn("Outdated metrics at time {}, last updated time {}", metrics.time(), broker.getTimestamp()); + return false; + } + for (Map.Entry entry : metrics.getMetricTypeValueMap().entrySet()) { + 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: + LOGGER.error("Unsupported broker metrics type {}", entry.getKey()); + break; + } + } + broker.setTimestamp(metrics.time()); + } finally { + lock.unlock(); + } + + 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(); + } + return broker; + } + + public Broker get(long timeSince) { + Broker broker; + lock.lock(); + try { + 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(); + } + return broker; + } + + public int id() { + return this.broker.getBrokerId(); + } + + public void setActive(boolean active) { + lock.lock(); + try { + this.broker.setActive(active); + } finally { + lock.unlock(); + } + } + + +} diff --git a/core/src/main/java/kafka/autobalancer/model/ClusterModel.java b/core/src/main/java/kafka/autobalancer/model/ClusterModel.java new file mode 100644 index 0000000000..3dd9942a1e --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/ClusterModel.java @@ -0,0 +1,336 @@ +/* + * 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 kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.listeners.BrokerStatusListener; +import kafka.autobalancer.listeners.TopicPartitionStatusListener; +import kafka.autobalancer.metricsreporter.metric.BrokerMetrics; +import kafka.autobalancer.metricsreporter.metric.TopicPartitionMetrics; +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RemoveTopicRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.metadata.BrokerRegistrationFencingChange; +import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange; +import org.slf4j.Logger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +public class ClusterModel implements BrokerStatusListener, TopicPartitionStatusListener { + private final Logger logger; + private static final String DEFAULT_RACK_ID = "rack_default"; + + /* + * Guard the change on cluster structure (add/remove for brokers, replicas) + */ + private final Lock clusterLock = new ReentrantLock(); + + /* cluster structure indices*/ + private final Map brokerIdToRackMap = new HashMap<>(); + private final Map brokerMap = new HashMap<>(); + private final Map> brokerReplicaMap = new HashMap<>(); + private final Map idToTopicNameMap = new HashMap<>(); + private final Map> topicPartitionReplicaMap = new HashMap<>(); + + private final long maxToleratedMetricsDelay; + private final boolean aggregateBrokerLoad; + + public ClusterModel(AutoBalancerControllerConfig config) { + this(config, null); + } + + public ClusterModel(AutoBalancerControllerConfig config, LogContext logContext) { + if (logContext == null) { + logContext = new LogContext("[ClusterModel]"); + } + logger = logContext.logger(ClusterModel.class); + maxToleratedMetricsDelay = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS); + aggregateBrokerLoad = config.getBoolean(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION); + } + + public ClusterModelSnapshot snapshot() { + return snapshot(Collections.emptySet(), Collections.emptySet()); + } + + public ClusterModelSnapshot snapshot(Set excludedBrokerIds, Set excludedTopics) { + ClusterModelSnapshot snapshot = new ClusterModelSnapshot(); + clusterLock.lock(); + try { + long now = System.currentTimeMillis(); + for (BrokerUpdater brokerUpdater : brokerMap.values()) { + BrokerUpdater.Broker broker = brokerUpdater.get(now - maxToleratedMetricsDelay); + if (broker == null) { + continue; + } + if (excludedBrokerIds.contains(broker.getBrokerId())) { + continue; + } + snapshot.addBroker(brokerIdToRackMap.get(broker.getBrokerId()), 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); + if (replica == null) { + continue; + } + if (excludedTopics.contains(replica.getTopicPartition().topic())) { + continue; + } + snapshot.addTopicPartition(brokerId, replica); + } + } + } finally { + clusterLock.unlock(); + } + + if (aggregateBrokerLoad) { + snapshot.aggregate(); + } + + return snapshot; + } + + public boolean updateBroker(BrokerMetrics brokerMetrics) { + BrokerUpdater brokerUpdater = null; + clusterLock.lock(); + try { + brokerUpdater = brokerMap.get(brokerMetrics.brokerId()); + } finally { + clusterLock.unlock(); + } + if (brokerUpdater != null) { + return brokerUpdater.update(brokerMetrics); + } + return false; + } + + public boolean updateTopicPartition(TopicPartitionMetrics topicPartitionMetrics) { + TopicPartitionReplicaUpdater replicaUpdater = null; + clusterLock.lock(); + try { + Map replicaMap = brokerReplicaMap.get(topicPartitionMetrics.brokerId()); + if (replicaMap != null) { + replicaUpdater = replicaMap.get(new TopicPartition(topicPartitionMetrics.topic(), topicPartitionMetrics.partition())); + } + } finally { + clusterLock.unlock(); + } + if (replicaUpdater != null) { + return replicaUpdater.update(topicPartitionMetrics); + } + return false; + } + + @Override + public void onBrokerRegister(RegisterBrokerRecord record) { + clusterLock.lock(); + try { + if (brokerMap.containsKey(record.brokerId())) { + return; + } + String rackId = StringUtils.isEmpty(record.rack()) ? DEFAULT_RACK_ID : record.rack(); + BrokerUpdater brokerUpdater = new BrokerUpdater(record.brokerId()); + brokerUpdater.setActive(true); + brokerIdToRackMap.putIfAbsent(record.brokerId(), rackId); + brokerMap.putIfAbsent(record.brokerId(), brokerUpdater); + brokerReplicaMap.put(record.brokerId(), new HashMap<>()); + } finally { + clusterLock.unlock(); + } + } + + @Override + public void onBrokerUnregister(UnregisterBrokerRecord record) { + clusterLock.lock(); + try { + brokerIdToRackMap.remove(record.brokerId()); + brokerMap.remove(record.brokerId()); + brokerReplicaMap.remove(record.brokerId()); + } finally { + clusterLock.unlock(); + } + } + + @Override + public void onBrokerRegistrationChanged(BrokerRegistrationChangeRecord record) { + BrokerUpdater brokerUpdater; + clusterLock.lock(); + try { + if (!brokerMap.containsKey(record.brokerId())) { + return; + } + brokerUpdater = brokerMap.get(record.brokerId()); + } finally { + clusterLock.unlock(); + } + if (brokerUpdater != null) { + brokerUpdater.setActive(record.fenced() != BrokerRegistrationFencingChange.FENCE.value() + && record.inControlledShutdown() != BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()); + } + } + + @Override + public void onTopicCreate(TopicRecord record) { + clusterLock.lock(); + try { + idToTopicNameMap.putIfAbsent(record.topicId(), record.name()); + topicPartitionReplicaMap.putIfAbsent(record.name(), new HashMap<>()); + } finally { + clusterLock.unlock(); + } + } + + @Override + public void onTopicDelete(RemoveTopicRecord record) { + clusterLock.lock(); + try { + String topicName = idToTopicNameMap.get(record.topicId()); + if (topicName == null) { + return; + } + idToTopicNameMap.remove(record.topicId()); + for (Map.Entry entry : topicPartitionReplicaMap.get(topicName).entrySet()) { + int partitionId = entry.getKey(); + int brokerId = entry.getValue(); + Map replicaMap = brokerReplicaMap.get(brokerId); + if (replicaMap != null) { + replicaMap.remove(new TopicPartition(topicName, partitionId)); + } + } + topicPartitionReplicaMap.remove(topicName); + } finally { + clusterLock.unlock(); + } + } + + @Override + public void onPartitionCreate(PartitionRecord record) { + clusterLock.lock(); + try { + String topicName = idToTopicNameMap.get(record.topicId()); + if (topicName == null) { + return; + } + if (record.replicas().size() != 1) { + logger.error("Illegal replica size {} for {}-{}", record.replicas().size(), topicName, record.partitionId()); + return; + } + if (!topicPartitionReplicaMap.containsKey(topicName)) { + logger.error("Create partition on invalid topic {}", topicName); + return; + } + int brokerIdToCreateOn = record.replicas().iterator().next(); + if (!brokerMap.containsKey(brokerIdToCreateOn)) { + logger.error("Create partition for topic {} on invalid broker {}", topicName, brokerIdToCreateOn); + return; + } + topicPartitionReplicaMap.get(topicName).put(record.partitionId(), brokerIdToCreateOn); + TopicPartition tp = new TopicPartition(topicName, record.partitionId()); + brokerReplicaMap.get(brokerIdToCreateOn).put(tp, new TopicPartitionReplicaUpdater(tp)); + } finally { + clusterLock.unlock(); + } + } + + @Override + public void onPartitionChange(PartitionChangeRecord record) { + clusterLock.lock(); + try { + String topicName = idToTopicNameMap.get(record.topicId()); + if (topicName == null) { + return; + } + + if (record.replicas() == null || record.replicas().size() != 1) { + return; + } + if (!topicPartitionReplicaMap.containsKey(topicName)) { + logger.error("Reassign partition {} on invalid topic {}", record.partitionId(), topicName); + return; + } + int brokerIdToReassign = record.replicas().iterator().next(); + if (!brokerMap.containsKey(brokerIdToReassign)) { + logger.error("Reassign partition {} for topic {} on invalid broker {}", record.partitionId(), topicName, brokerIdToReassign); + return; + } + int oldBrokerId = topicPartitionReplicaMap.get(topicName).getOrDefault(record.partitionId(), -1); + if (oldBrokerId == brokerIdToReassign) { + logger.warn("Reassign partition {} for topic {} on same broker {}, {}", record.partitionId(), topicName, oldBrokerId, record); + return; + } + if (oldBrokerId != -1) { + TopicPartition tp = new TopicPartition(topicName, record.partitionId()); + TopicPartitionReplicaUpdater replicaUpdater = brokerReplicaMap.get(oldBrokerId).get(tp); + brokerReplicaMap.get(brokerIdToReassign).put(tp, replicaUpdater); + brokerReplicaMap.get(oldBrokerId).remove(tp); + } + topicPartitionReplicaMap.get(topicName).put(record.partitionId(), brokerIdToReassign); + } finally { + clusterLock.unlock(); + } + } + + public BrokerUpdater brokerUpdater(int brokerId) { + clusterLock.lock(); + try { + return brokerMap.get(brokerId); + } finally { + clusterLock.unlock(); + } + } + + public TopicPartitionReplicaUpdater replicaUpdater(int brokerId, TopicPartition tp) { + clusterLock.lock(); + try { + if (!brokerReplicaMap.containsKey(brokerId)) { + return null; + } + return brokerReplicaMap.get(brokerId).get(tp); + } finally { + clusterLock.unlock(); + } + } + + public String topicName(Uuid topicId) { + clusterLock.lock(); + try { + return idToTopicNameMap.get(topicId); + } finally { + clusterLock.unlock(); + } + } + /* Code visible for test end*/ +} diff --git a/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java b/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java new file mode 100644 index 0000000000..86689d44c6 --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/ClusterModelSnapshot.java @@ -0,0 +1,105 @@ +/* + * 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 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; +import java.util.HashMap; +import java.util.Map; + +public class ClusterModelSnapshot { + + private final Map rackToBrokerMap; + private final Map brokerMap; + private final Map> brokerToReplicaMap; + + public ClusterModelSnapshot() { + rackToBrokerMap = new HashMap<>(); + brokerMap = new HashMap<>(); + brokerToReplicaMap = new HashMap<>(); + } + + public void aggregate() { + // Override broker load with sum of replicas + 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(); + brokerMap.get(brokerId).setLoad(resource, sum); + } + } + } + + 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 addTopicPartition(int brokerId, TopicPartitionReplica replica) { + brokerToReplicaMap.putIfAbsent(brokerId, new HashMap<>()); + brokerToReplicaMap.get(brokerId).put(replica.getTopicPartition(), replica); + } + + public Broker broker(int brokerId) { + return brokerMap.get(brokerId); + } + + public Collection brokers() { + return brokerMap.values(); + } + + public TopicPartitionReplica replica(int brokerId, TopicPartition tp) { + if (!brokerToReplicaMap.containsKey(brokerId)) { + return null; + } + if (!brokerToReplicaMap.get(brokerId).containsKey(tp)) { + return null; + } + return brokerToReplicaMap.get(brokerId).get(tp); + } + + 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()); + if (srcBroker == null || destBroker == null) { + return; + } + 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()); + 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 new file mode 100644 index 0000000000..6197b9816f --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/ModelUtils.java @@ -0,0 +1,32 @@ +/* + * 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 kafka.autobalancer.common.Resource; + +public class ModelUtils { + + 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); + } + } + +} diff --git a/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java b/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java new file mode 100644 index 0000000000..e451e0398b --- /dev/null +++ b/core/src/main/java/kafka/autobalancer/model/TopicPartitionReplicaUpdater.java @@ -0,0 +1,182 @@ +/* + * 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 kafka.autobalancer.common.Resource; +import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics; +import kafka.autobalancer.metricsreporter.metric.MetricsUtils; +import kafka.autobalancer.metricsreporter.metric.RawMetricType; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +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 TopicPartitionReplicaUpdater { + private static final Logger LOGGER = LoggerFactory.getLogger(TopicPartitionReplicaUpdater.class); + private final Lock lock = new ReentrantLock(); + private final TopicPartitionReplica replica; + + public TopicPartitionReplicaUpdater(TopicPartition tp) { + this.replica = new TopicPartitionReplica(tp); + } + + public static class TopicPartitionReplica { + private final TopicPartition tp; + private final double[] replicaLoad = new double[Resource.cachedValues().size()]; + private final Set resources = new HashSet<>(); + private long timestamp; + + public TopicPartitionReplica(TopicPartition tp) { + this.tp = tp; + } + + public TopicPartitionReplica(TopicPartitionReplica other) { + this.tp = new TopicPartition(other.tp.topic(), other.tp.partition()); + System.arraycopy(other.replicaLoad, 0, this.replicaLoad, 0, other.replicaLoad.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.replicaLoad[resource.id()] = value; + } + + public double load(Resource resource) { + if (!this.resources.contains(resource)) { + return 0.0; + } + return this.replicaLoad[resource.id()]; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public long getTimestamp() { + return timestamp; + } + + public TopicPartition getTopicPartition() { + return tp; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TopicPartitionReplica replica = (TopicPartitionReplica) o; + return tp.equals(replica.tp); + } + + @Override + public int hashCode() { + return Objects.hashCode(tp); + } + + @Override + public String toString() { + return "TopicPartitionReplica{" + + "tp=" + tp + + ", replicaLoad=" + Arrays.toString(replicaLoad) + + '}'; + } + } + + public boolean update(AutoBalancerMetrics metrics) { + if (metrics.metricClassId() != AutoBalancerMetrics.MetricClassId.PARTITION_METRIC) { + LOGGER.error("Mismatched metrics type {} for broker", metrics.metricClassId()); + return false; + } + + if (!MetricsUtils.sanityCheckTopicPartitionMetricsCompleteness(metrics)) { + LOGGER.error("Topic partition metrics sanity check failed, metrics is incomplete {}", metrics); + return false; + } + + lock.lock(); + try { + if (metrics.time() < this.replica.getTimestamp()) { + LOGGER.warn("Outdated metrics at time {}, last updated time {}", metrics.time(), this.replica.getTimestamp()); + return false; + } + for (Map.Entry entry : metrics.getMetricTypeValueMap().entrySet()) { + 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; + case PARTITION_SIZE: + // simply update the timestamp + break; + default: + LOGGER.error("Unsupported broker metrics type {}", entry.getKey()); + } + } + this.replica.setTimestamp(metrics.time()); + } finally { + lock.unlock(); + } + 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(); + } + return replica; + } + + public TopicPartitionReplica get(long timeSince) { + TopicPartitionReplica replica; + lock.lock(); + try { + if (this.replica.timestamp < timeSince) { + LOGGER.warn("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(); + } + return replica; + } + + public TopicPartition topicPartition() { + return this.replica.getTopicPartition(); + } +} diff --git a/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java b/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java new file mode 100644 index 0000000000..d4d5fbe84e --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/AutoBalancerManagerTest.java @@ -0,0 +1,306 @@ +/* + * 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; + +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.metricsreporter.AutoBalancerMetricsReporter; +import kafka.autobalancer.utils.AbstractLoadGenerator; +import kafka.autobalancer.utils.AutoBalancerClientsIntegrationTestHarness; +import kafka.autobalancer.utils.ConsumeLoadGenerator; +import kafka.autobalancer.utils.ProduceLoadGenerator; +import kafka.server.KafkaConfig; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.StringJoiner; + +@Tag("esUnit") +public class AutoBalancerManagerTest extends AutoBalancerClientsIntegrationTestHarness { + /** + * Set up the unit test. + */ + @BeforeEach + public void setUp() { + super.setUp(); + } + + @AfterEach + public void tearDown() { + super.tearDown(); + } + + @Override + protected Map overridingNodeProps() { + Map props = new HashMap<>(); + props.put(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG, METRIC_TOPIC); + props.put(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG, "1"); + props.put(KafkaConfig.LogFlushIntervalMessagesProp(), "1"); + props.put(KafkaConfig.OffsetsTopicReplicationFactorProp(), "1"); + props.put(KafkaConfig.DefaultReplicationFactorProp(), "1"); + props.put(KafkaConfig.OffsetsTopicPartitionsProp(), "1"); + return props; + } + + @Override + 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; + } + + @Override + 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"); + 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"); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS, "10000"); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_LOAD_AGGREGATION, "true"); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXCLUDE_TOPICS, "__consumer_offsets," + METRIC_TOPIC); + + return props; + } + + private Properties createProducerProperties() { + long randomToken = TestUtils.RANDOM.nextLong(); + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "ab_producer-" + randomToken); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + return getProducerProperties(producerProps); + } + + private Properties createConsumerProperties() { + long randomToken = TestUtils.RANDOM.nextLong(); + Properties consumerProps = new Properties(); + consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "ab_consumer-" + randomToken); + consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "ab-group-" + randomToken); + consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + consumerProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + return consumerProps; + } + + @Disabled + @Test + public void testShutdown() throws Exception { + createTopic(TOPIC_1, Map.of(0, List.of(0), 1, List.of(0))); + List producers = new ArrayList<>(); + List consumers = new ArrayList<>(); + // broker-0: in: 60, out: 60 + AbstractLoadGenerator topic1Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 0, 4); + AbstractLoadGenerator topic1Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 0); + producers.add(topic1Partition0Producer); + consumers.add(topic1Partition0Consumer); + + // broker-1: in: 60, out: 60 + AbstractLoadGenerator topic1Partition1Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 1, 4); + AbstractLoadGenerator topic1Partition1Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 1); + producers.add(topic1Partition1Producer); + consumers.add(topic1Partition1Consumer); + + consumers.forEach(AbstractLoadGenerator::start); + producers.forEach(AbstractLoadGenerator::start); + + Thread.sleep(5000); + + Assertions.assertTimeout(Duration.ofMillis(5000), () -> producers.forEach(AbstractLoadGenerator::shutdown)); + Assertions.assertTimeout(Duration.ofMillis(5000), () -> consumers.forEach(AbstractLoadGenerator::shutdown)); + + System.out.printf("Completed%n"); + } + + @Disabled + @Test + public void testMoveActionBalance() throws Exception { + createTopic(TOPIC_1, Map.of(0, List.of(0), 1, List.of(1))); + createTopic(TOPIC_2, Map.of(0, List.of(0))); + List producers = new ArrayList<>(); + List consumers = new ArrayList<>(); + // broker-0: in: 90, out: 90 + AbstractLoadGenerator topic1Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 0, 40); + AbstractLoadGenerator topic1Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 0); + producers.add(topic1Partition0Producer); + consumers.add(topic1Partition0Consumer); + + AbstractLoadGenerator topic2Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_2, 0, 50); + AbstractLoadGenerator topic2Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_2, 0); + producers.add(topic2Partition0Producer); + consumers.add(topic2Partition0Consumer); + + // broker-1: in: 10, out: 10 + AbstractLoadGenerator topic1Partition1Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 1, 10); + AbstractLoadGenerator topic1Partition1Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 1); + producers.add(topic1Partition1Producer); + consumers.add(topic1Partition1Consumer); + + consumers.forEach(AbstractLoadGenerator::start); + producers.forEach(AbstractLoadGenerator::start); + TestUtils.waitForCondition(() -> getReplicaFor(TOPIC_2, 0) == 1, 30000L, 1000L, + () -> "failed to reassign"); + producers.forEach(AbstractLoadGenerator::shutdown); + consumers.forEach(AbstractLoadGenerator::shutdown); + + // check only for msg failed here, leave message integration check to E2E test + Assertions.assertEquals(0, topic1Partition0Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition0Consumer.msgFailed()); + + Assertions.assertEquals(0, topic1Partition1Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition1Consumer.msgFailed()); + + Assertions.assertEquals(0, topic2Partition0Producer.msgFailed()); + Assertions.assertEquals(0, topic2Partition0Consumer.msgFailed()); + } + + @Disabled + @Test + public void testBrokerShutdown() throws Exception { + createTopic(TOPIC_1, Map.of(0, List.of(0), 1, List.of(1))); + List producers = new ArrayList<>(); + List consumers = new ArrayList<>(); + // broker-0: in: 60, out: 60 + AbstractLoadGenerator topic1Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 0, 4); + AbstractLoadGenerator topic1Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 0); + producers.add(topic1Partition0Producer); + consumers.add(topic1Partition0Consumer); + + // broker-1: in: 60, out: 60 + AbstractLoadGenerator topic1Partition1Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 1, 4); + AbstractLoadGenerator topic1Partition1Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 1); + producers.add(topic1Partition1Producer); + consumers.add(topic1Partition1Consumer); + + consumers.forEach(AbstractLoadGenerator::start); + producers.forEach(AbstractLoadGenerator::start); + + System.out.printf("Shutdown broker-1%n"); + Assertions.assertTimeout(Duration.ofMillis(10000), () -> cluster.brokers().get(1).shutdown()); + System.out.printf("Broker-1 shutdown complete%n"); + TestUtils.waitForCondition(() -> { + Optional partitionInfoOptional1 = getPartitionInfoFor(TOPIC_1, 1); + Optional partitionInfoOptional2 = getPartitionInfoFor(TOPIC_1, 0); + if (partitionInfoOptional1.isEmpty() || partitionInfoOptional2.isEmpty()) { + return false; + } + TopicPartitionInfo partitionInfo1 = partitionInfoOptional1.get(); + TopicPartitionInfo partitionInfo2 = partitionInfoOptional2.get(); + if (partitionInfo1.replicas().isEmpty() || partitionInfo2.replicas().isEmpty()) { + return false; + } + return partitionInfo1.leader().id() == 0 && partitionInfo2.leader().id() == 0; + }, 30000L, 1000L, () -> "failed to reassign replicas from offline broker"); + + producers.forEach(AbstractLoadGenerator::shutdown); + consumers.forEach(AbstractLoadGenerator::shutdown); + + Assertions.assertEquals(0, topic1Partition0Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition0Consumer.msgFailed()); + + Assertions.assertEquals(0, topic1Partition1Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition1Consumer.msgFailed()); + + System.out.printf("Complete%n"); + } + + @Disabled + @Test + public void testControllerShutdown() throws Exception { + createTopic(TOPIC_1, Map.of(0, List.of(0), 1, List.of(1))); + createTopic(TOPIC_2, Map.of(0, List.of(0))); + List producers = new ArrayList<>(); + List consumers = new ArrayList<>(); + // broker-0: in: 90, out: 90 + AbstractLoadGenerator topic1Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 0, 40); + AbstractLoadGenerator topic1Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 0); + producers.add(topic1Partition0Producer); + consumers.add(topic1Partition0Consumer); + + AbstractLoadGenerator topic2Partition0Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_2, 0, 50); + AbstractLoadGenerator topic2Partition0Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_2, 0); + producers.add(topic2Partition0Producer); + consumers.add(topic2Partition0Consumer); + + // broker-1: in: 10, out: 10 + AbstractLoadGenerator topic1Partition1Producer = new ProduceLoadGenerator(createProducerProperties(), TOPIC_1, 1, 10); + AbstractLoadGenerator topic1Partition1Consumer = new ConsumeLoadGenerator(createConsumerProperties(), TOPIC_1, 1); + producers.add(topic1Partition1Producer); + consumers.add(topic1Partition1Consumer); + + consumers.forEach(AbstractLoadGenerator::start); + producers.forEach(AbstractLoadGenerator::start); + + int leaderId = cluster.raftManagers().values().iterator().next().client().leaderAndEpoch().leaderId().getAsInt(); + System.out.printf("Shutdown controller-%d%n", leaderId); + Assertions.assertTimeout(Duration.ofMillis(10000), () -> cluster.controllers().get(leaderId).shutdown()); + System.out.printf("Controller-%d shutdown complete%n", leaderId); + + TestUtils.waitForCondition(() -> getReplicaFor(TOPIC_2, 0) == 1, 30000L, 1000L, + () -> "failed to reassign"); + producers.forEach(AbstractLoadGenerator::shutdown); + consumers.forEach(AbstractLoadGenerator::shutdown); + + Assertions.assertEquals(0, topic1Partition0Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition0Consumer.msgFailed()); + + Assertions.assertEquals(0, topic1Partition1Producer.msgFailed()); + Assertions.assertEquals(0, topic1Partition1Consumer.msgFailed()); + + Assertions.assertEquals(0, topic2Partition0Producer.msgFailed()); + Assertions.assertEquals(0, topic2Partition0Consumer.msgFailed()); + } +} diff --git a/core/src/test/java/kafka/autobalancer/ExecutionManagerTest.java b/core/src/test/java/kafka/autobalancer/ExecutionManagerTest.java new file mode 100644 index 0000000000..c6ca31640c --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/ExecutionManagerTest.java @@ -0,0 +1,102 @@ +/* + * 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; + +import kafka.autobalancer.common.Action; +import kafka.autobalancer.common.ActionType; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.test.MockController; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; +import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.ControllerRequestContext; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +@Tag("esUnit") +public class ExecutionManagerTest { + + private boolean checkTopicPartition(AlterPartitionReassignmentsRequestData.ReassignableTopic topic, + String name, int partitionId, int nodeId) { + if (!topic.name().equals(name)) { + return false; + } + if (topic.partitions().size() != 1) { + return false; + } + AlterPartitionReassignmentsRequestData.ReassignablePartition partition = topic.partitions().get(0); + if (partition.partitionIndex() != partitionId) { + return false; + } + return partition.replicas().size() == 1 && partition.replicas().get(0) == nodeId; + } + + @Test + public void testExecuteActions() throws Exception { + Map props = new HashMap<>(); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_EXECUTION_INTERVAL_MS, 100L); + AutoBalancerControllerConfig config = new AutoBalancerControllerConfig(props, false); + + Controller controller = Mockito.mock(MockController.class); + + final ArgumentCaptor ctxCaptor = ArgumentCaptor.forClass(ControllerRequestContext.class); + final ArgumentCaptor reqCaptor = ArgumentCaptor.forClass(AlterPartitionReassignmentsRequestData.class); + + Mockito.doAnswer(answer -> CompletableFuture.completedFuture(new AlterPartitionReassignmentsResponseData())) + .when(controller).alterPartitionReassignments(ctxCaptor.capture(), reqCaptor.capture()); + + ExecutionManager executionManager = new ExecutionManager(config, controller); + executionManager.start(); + + List actionList = List.of( + new Action(ActionType.MOVE, new TopicPartition("topic1", 0), 0, 1), + new Action(ActionType.SWAP, new TopicPartition("topic2", 0), 0, 1, new TopicPartition("topic1", 1))); + executionManager.appendActions(actionList); + + TestUtils.waitForCondition(() -> { + List reqs = reqCaptor.getAllValues(); + if (reqs.size() != 2) { + return false; + } + AlterPartitionReassignmentsRequestData reqMove = reqs.get(0); + if (reqMove.topics().size() != 1) { + return false; + } + if (!checkTopicPartition(reqMove.topics().get(0), "topic1", 0, 1)) { + return false; + } + AlterPartitionReassignmentsRequestData reqSwap = reqs.get(1); + if (reqSwap.topics().size() != 2) { + return false; + } + return checkTopicPartition(reqSwap.topics().get(0), "topic2", 0, 1) + && checkTopicPartition(reqSwap.topics().get(1), "topic1", 1, 0); + }, 5000L, 1000L, () -> "failed to meet reassign"); + + executionManager.shutdown(); + } +} diff --git a/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java b/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java new file mode 100644 index 0000000000..0a1791be62 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/LoadRetrieverTest.java @@ -0,0 +1,206 @@ +/* + * 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; + +import kafka.autobalancer.common.Resource; +import kafka.autobalancer.config.AutoBalancerConfig; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; +import kafka.autobalancer.metricsreporter.AutoBalancerMetricsReporter; +import kafka.autobalancer.model.ClusterModel; +import kafka.autobalancer.model.ClusterModelSnapshot; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import kafka.autobalancer.utils.AutoBalancerClientsIntegrationTestHarness; +import kafka.cluster.EndPoint; +import kafka.server.BrokerServer; +import kafka.server.KafkaConfig; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.AfterEach; +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.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Tag("esUnit") +public class LoadRetrieverTest extends AutoBalancerClientsIntegrationTestHarness { + protected static final String METRIC_TOPIC = "AutoBalancerMetricsReporterTest"; + + /** + * Setup the unit test. + */ + @BeforeEach + public void setUp() { + super.setUp(); + } + + @AfterEach + public void tearDown() { + super.tearDown(); + } + + @Override + protected Map overridingNodeProps() { + Map props = new HashMap<>(); + props.put(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG, METRIC_TOPIC); + props.put(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG, "1"); + props.put(KafkaConfig.LogFlushIntervalMessagesProp(), "1"); + props.put(KafkaConfig.OffsetsTopicReplicationFactorProp(), "1"); + props.put(KafkaConfig.DefaultReplicationFactorProp(), "1"); + return props; + } + + @Override + 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"); + + return props; + } + + private boolean checkConsumeRecord(ClusterModel clusterModel, int brokerId) { + ClusterModelSnapshot snapshot = clusterModel.snapshot(); + 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); + if (testReplica == null || metricReplica == null) { + return false; + } + + return testReplica.load(Resource.NW_IN) != 0 + && testReplica.load(Resource.NW_OUT) == 0 + && metricReplica.load(Resource.NW_IN) != 0 + && metricReplica.load(Resource.NW_OUT) != 0; + } + + @Test + public void testLoadRetrieverShutdown() { + Map props = new HashMap<>(); + AutoBalancerControllerConfig config = new AutoBalancerControllerConfig(props, false); + ClusterModel clusterModel = new ClusterModel(config); + LoadRetriever loadRetriever = new LoadRetriever(config, + cluster.controllers().values().iterator().next().controller(), clusterModel); + loadRetriever.start(); + + Assertions.assertTimeout(Duration.ofMillis(5000), loadRetriever::shutdown); + + LoadRetriever loadRetriever2 = new LoadRetriever(config, + cluster.controllers().values().iterator().next().controller(), clusterModel); + loadRetriever2.start(); + BrokerServer broker = cluster.brokers().values().iterator().next(); + KafkaConfig brokerConfig = broker.config(); + EndPoint endpoint = brokerConfig.effectiveAdvertisedListeners().iterator().next(); + RegisterBrokerRecord record = new RegisterBrokerRecord() + .setBrokerId(brokerConfig.brokerId()) + .setEndPoints(new RegisterBrokerRecord.BrokerEndpointCollection( + Collections.singletonList(new RegisterBrokerRecord.BrokerEndpoint() + .setName("PLAINTEXT") + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setSecurityProtocol(endpoint.securityProtocol().id)).iterator())); + loadRetriever2.onBrokerRegister(record); + Assertions.assertTimeout(Duration.ofMillis(5000), loadRetriever2::shutdown); + } + + @Test + public void testConsume() throws InterruptedException { + Map props = new HashMap<>(); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_TOPIC_CONFIG, METRIC_TOPIC); + props.put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ACCEPTED_METRICS_DELAY_MS, 3000L); + AutoBalancerControllerConfig config = new AutoBalancerControllerConfig(props, false); + + ClusterModel clusterModel = new ClusterModel(config); + LoadRetriever loadRetriever = new LoadRetriever(config, + cluster.controllers().values().iterator().next().controller(), clusterModel); + loadRetriever.start(); + + BrokerServer broker = cluster.brokers().values().iterator().next(); + KafkaConfig brokerConfig = broker.config(); + EndPoint endpoint = brokerConfig.effectiveAdvertisedListeners().iterator().next(); + RegisterBrokerRecord record = new RegisterBrokerRecord() + .setBrokerId(brokerConfig.brokerId()) + .setEndPoints(new RegisterBrokerRecord.BrokerEndpointCollection( + Collections.singletonList(new RegisterBrokerRecord.BrokerEndpoint() + .setName("PLAINTEXT") + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setSecurityProtocol(endpoint.securityProtocol().id)).iterator())) + .setFenced(false) + .setInControlledShutdown(false); + clusterModel.onBrokerRegister(record); + Uuid testTopicId = Uuid.randomUuid(); + Uuid metricTopicId = Uuid.randomUuid(); + clusterModel.onTopicCreate(new TopicRecord() + .setName(TOPIC_0) + .setTopicId(testTopicId)); + clusterModel.onPartitionCreate(new PartitionRecord() + .setReplicas(List.of(brokerConfig.brokerId())) + .setTopicId(testTopicId) + .setPartitionId(0)); + clusterModel.onTopicCreate(new TopicRecord() + .setName(METRIC_TOPIC) + .setTopicId(metricTopicId)); + clusterModel.onPartitionCreate(new PartitionRecord() + .setReplicas(List.of(brokerConfig.brokerId())) + .setTopicId(metricTopicId) + .setPartitionId(0)); + loadRetriever.onBrokerRegister(record); + + TestUtils.waitForCondition(() -> checkConsumeRecord(clusterModel, brokerConfig.brokerId()), + 15000L, 1000L, () -> "cluster model failed to reach expected status"); + + UnregisterBrokerRecord unregisterRecord = new UnregisterBrokerRecord() + .setBrokerId(brokerConfig.brokerId()); + loadRetriever.onBrokerUnregister(unregisterRecord); + Thread.sleep(5000); + Assertions.assertTrue(() -> { + ClusterModelSnapshot snapshot = clusterModel.snapshot(); + if (snapshot.broker(brokerConfig.brokerId()) != null) { + return false; + } + TopicPartitionReplicaUpdater.TopicPartitionReplica testReplica = snapshot.replica(brokerConfig.brokerId(), + new TopicPartition(TOPIC_0, 0)); + TopicPartitionReplicaUpdater.TopicPartitionReplica metricReplica = snapshot.replica(brokerConfig.brokerId(), + new TopicPartition(METRIC_TOPIC, 0)); + return testReplica == null && metricReplica == null; + }); + + clusterModel.onBrokerRegister(record); + loadRetriever.onBrokerRegister(record); + TestUtils.waitForCondition(() -> checkConsumeRecord(clusterModel, brokerConfig.brokerId()), + 15000L, 1000L, () -> "cluster model failed to reach expected status"); + Assertions.assertTimeout(Duration.ofMillis(5000), loadRetriever::shutdown); + } +} diff --git a/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java b/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java new file mode 100644 index 0000000000..43dd74aba6 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/goals/AbstractGoalTest.java @@ -0,0 +1,163 @@ +/* + * 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.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.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.StringJoiner; + +@Tag("esUnit") +public class AbstractGoalTest 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()) + .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); + 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); + List goalList = controllerConfig.getConfiguredInstances(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, AbstractGoal.class); + goalList.sort(Comparator.reverseOrder()); + for (AbstractGoal goal : goalList) { + goalMap.put(goal.name(), goal); + } + } + + @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 (AbstractGoal 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 (AbstractGoal 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)); + } + } +} diff --git a/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java new file mode 100644 index 0000000000..e77f075743 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceCapacityGoalTest.java @@ -0,0 +1,373 @@ +/* + * 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("esUnit") +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 AbstractGoal getGoalByResource(Resource resource) { + AbstractGoal 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) { + AbstractGoal 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) { + AbstractGoal 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) { + AbstractGoal 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() { + AbstractGoal 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/AbstractResourceDistributionGoalTest.java new file mode 100644 index 0000000000..d431786f2e --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/goals/AbstractResourceDistributionGoalTest.java @@ -0,0 +1,308 @@ +/* + * 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("esUnit") +public class AbstractResourceDistributionGoalTest 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); + 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); + List goalList = controllerConfig.getConfiguredInstances(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_GOALS, AbstractGoal.class); + for (AbstractGoal goal : goalList) { + goalMap.put(goal.name(), goal); + } + } + + private AbstractGoal getGoalByResource(Resource resource) { + AbstractGoal goal = null; + switch (resource) { + case NW_IN: + goal = goalMap.get(NetworkInDistributionGoal.class.getSimpleName()); + break; + case NW_OUT: + goal = goalMap.get(NetworkOutDistributionGoal.class.getSimpleName()); + break; + default: + break; + } + return goal; + } + + private void testActionAcceptanceScore(Resource resource) { + AbstractGoal 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) { + AbstractGoal 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 = 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); + 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); + replica4.setLoad(resource, 15); + replica5.setLoad(resource, 5); + 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)); + } + } + + private void testSingleResourceDistributionOptimizeMultiMoveOut(Resource resource) { + AbstractGoal 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 = 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); + replica0.setLoad(resource, 10); + replica1.setLoad(resource, 20); + replica2.setLoad(resource, 15); + replica3.setLoad(resource, 5); + replica4.setLoad(resource, 5); + replica5.setLoad(resource, 10); + 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); + replica7.setLoad(resource, 5); + replica8.setLoad(resource, 5); + 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)); + } + } + + private void testSingleResourceDistributionOptimizeMultiMoveIn(Resource resource) { + AbstractGoal 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 = 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); + 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); + replica3.setLoad(resource, 10); + replica4.setLoad(resource, 20); + replica5.setLoad(resource, 15); + replica6.setLoad(resource, 5); + replica7.setLoad(resource, 5); + replica8.setLoad(resource, 10); + replica9.setLoad(resource, 15); + 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 testSingleResourceDistributionOptimizeOneMove() { + testSingleResourceDistributionOptimizeOneMove(Resource.NW_IN); + testSingleResourceDistributionOptimizeOneMove(Resource.NW_OUT); + } + + @Test + public void testSingleResourceDistributionOptimizeMultiMoveOut() { + testSingleResourceDistributionOptimizeMultiMoveOut(Resource.NW_IN); + testSingleResourceDistributionOptimizeMultiMoveOut(Resource.NW_OUT); + testSingleResourceDistributionOptimizeMultiMoveIn(Resource.NW_IN); + testSingleResourceDistributionOptimizeMultiMoveIn(Resource.NW_OUT); + } + + private void testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource resource) { + AbstractGoal 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); + + 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 testMultiGoalOptimizeWithOneToOneReplicaSwap() { + testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource.NW_IN); + testMultiGoalOptimizeWithOneToOneReplicaSwap(Resource.NW_OUT); + } + +} diff --git a/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java b/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java new file mode 100644 index 0000000000..8950619ff6 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/goals/GoalTestBase.java @@ -0,0 +1,52 @@ +/* + * 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.model.BrokerUpdater; +import kafka.autobalancer.model.ClusterModelSnapshot; +import kafka.autobalancer.model.TopicPartitionReplicaUpdater; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Tag; + +@Tag("esUnit") +public class GoalTestBase { + protected static final String RACK = "default"; + protected static final String TOPIC_0 = "TestTopic0"; + protected static final String TOPIC_1 = "TestTopic1"; + protected static final String TOPIC_2 = "TestTopic2"; + 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); + return broker; + } + + protected TopicPartitionReplicaUpdater.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); + return replica; + } +} diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java new file mode 100644 index 0000000000..c56f17056e --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/AutoBalancerMetricsReporterTest.java @@ -0,0 +1,138 @@ +/* + * 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; + +import kafka.autobalancer.config.AutoBalancerConfig; +import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; +import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics; +import kafka.autobalancer.metricsreporter.metric.MetricSerde; +import kafka.autobalancer.metricsreporter.metric.RawMetricType; +import kafka.autobalancer.utils.AutoBalancerClientsIntegrationTestHarness; +import kafka.server.KafkaConfig; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.junit.jupiter.api.AfterEach; +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.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.ALL_TOPIC_BYTES_IN; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.ALL_TOPIC_BYTES_OUT; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.BROKER_CAPACITY_NW_IN; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.BROKER_CAPACITY_NW_OUT; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.BROKER_CPU_UTIL; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.PARTITION_SIZE; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.TOPIC_PARTITION_BYTES_IN; +import static kafka.autobalancer.metricsreporter.metric.RawMetricType.TOPIC_PARTITION_BYTES_OUT; + +@Tag("esUnit") +public class AutoBalancerMetricsReporterTest extends AutoBalancerClientsIntegrationTestHarness { + + /** + * Setup the unit test. + */ + @BeforeEach + public void setUp() { + super.setUp(); + } + + @AfterEach + public void tearDown() { + super.tearDown(); + } + + @Override + protected Map overridingNodeProps() { + Map props = new HashMap<>(); + props.put(AutoBalancerConfig.AUTO_BALANCER_TOPIC_CONFIG, METRIC_TOPIC); + props.put(AutoBalancerConfig.AUTO_BALANCER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG, "1"); + props.put(KafkaConfig.LogFlushIntervalMessagesProp(), "1"); + props.put(KafkaConfig.OffsetsTopicReplicationFactorProp(), "1"); + props.put(KafkaConfig.DefaultReplicationFactorProp(), "1"); + return props; + } + + @Override + 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"); + return props; + } + + @Test + public void testReportingMetrics() { + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, MetricSerde.class.getName()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "testReportingMetrics"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + 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 metricTypes = new HashSet<>(); + ConsumerRecords records; + while (metricTypes.size() < (expectedBrokerMetricTypes.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()) { + 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(expectedMetricTypes, metricTypes, "Expected " + expectedMetricTypes + ", but saw " + metricTypes); + } +} diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java new file mode 100644 index 0000000000..6578109591 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricSerdeTest.java @@ -0,0 +1,79 @@ +/* + * 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; + +import kafka.autobalancer.metricsreporter.exception.UnknownVersionException; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@Tag("esUnit") +public class MetricSerdeTest { + private static final long TIME = 123L; + private static final int BROKER_ID = 0; + private static final String TOPIC = "topic"; + private static final int PARTITION = 100; + 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) + .put(RawMetricType.PARTITION_SIZE, VALUE) + .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(2, metricMap.size()); + assertTrue(metricMap.containsKey(RawMetricType.PARTITION_SIZE)); + assertTrue(metricMap.containsKey(RawMetricType.TOPIC_PARTITION_BYTES_IN)); + assertEquals(VALUE, metricMap.get(RawMetricType.PARTITION_SIZE), 0.000001); + assertEquals(VALUE1, metricMap.get(RawMetricType.TOPIC_PARTITION_BYTES_IN), 0.000001); + assertEquals(TIME, deserialized.time()); + assertEquals(BROKER_ID, deserialized.brokerId()); + assertEquals(TOPIC, ((TopicPartitionMetrics) deserialized).topic()); + assertEquals(PARTITION, ((TopicPartitionMetrics) deserialized).partition()); + assertEquals("", deserialized.brokerRack()); + } +} diff --git a/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java new file mode 100644 index 0000000000..41c802b1ba --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/metricsreporter/metric/MetricsUtilsTest.java @@ -0,0 +1,52 @@ +/* + * 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; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +@Tag("esUnit") +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); + metrics.put(RawMetricType.TOPIC_PARTITION_BYTES_IN, 10); + Assertions.assertFalse(MetricsUtils.sanityCheckTopicPartitionMetricsCompleteness(metrics)); + metrics.put(RawMetricType.TOPIC_PARTITION_BYTES_OUT, 10); + Assertions.assertFalse(MetricsUtils.sanityCheckTopicPartitionMetricsCompleteness(metrics)); + metrics.put(RawMetricType.PARTITION_SIZE, 10); + Assertions.assertTrue(MetricsUtils.sanityCheckTopicPartitionMetricsCompleteness(metrics)); + } +} diff --git a/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java b/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java new file mode 100644 index 0000000000..295b0dd367 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/model/ClusterModelTest.java @@ -0,0 +1,254 @@ +/* + * 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 kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.metricsreporter.metric.BrokerMetrics; +import kafka.autobalancer.metricsreporter.metric.RawMetricType; +import kafka.autobalancer.metricsreporter.metric.TopicPartitionMetrics; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RemoveTopicRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.List; + +@Tag("esUnit") +public class ClusterModelTest { + + @Test + public void testRegisterBroker() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + RegisterBrokerRecord record1 = new RegisterBrokerRecord() + .setBrokerId(1); + RegisterBrokerRecord record2 = new RegisterBrokerRecord() + .setBrokerId(2); + clusterModel.onBrokerRegister(record1); + clusterModel.onBrokerRegister(record2); + clusterModel.onBrokerRegister(record1); + + Assertions.assertEquals(1, clusterModel.brokerUpdater(1).id()); + Assertions.assertEquals(2, clusterModel.brokerUpdater(2).id()); + } + + @Test + public void testUnregisterBroker() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + RegisterBrokerRecord record1 = new RegisterBrokerRecord() + .setBrokerId(1); + RegisterBrokerRecord record2 = new RegisterBrokerRecord() + .setBrokerId(2); + clusterModel.onBrokerRegister(record1); + clusterModel.onBrokerRegister(record2); + clusterModel.onBrokerRegister(record1); + + Assertions.assertEquals(1, clusterModel.brokerUpdater(1).id()); + Assertions.assertEquals(2, clusterModel.brokerUpdater(2).id()); + + UnregisterBrokerRecord unregisterRecord = new UnregisterBrokerRecord() + .setBrokerId(2); + clusterModel.onBrokerUnregister(unregisterRecord); + + Assertions.assertEquals(1, clusterModel.brokerUpdater(1).id()); + Assertions.assertNull(clusterModel.brokerUpdater(2)); + } + + @Test + public void testCreateTopic() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + String topicName = "testTopic"; + Uuid topicId = Uuid.randomUuid(); + TopicRecord record = new TopicRecord() + .setName(topicName) + .setTopicId(topicId); + clusterModel.onTopicCreate(record); + + Assertions.assertEquals(topicName, clusterModel.topicName(topicId)); + } + + @Test + public void testDeleteTopic() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + String topicName = "testTopic"; + Uuid topicId = Uuid.randomUuid(); + TopicRecord record = new TopicRecord() + .setName(topicName) + .setTopicId(topicId); + clusterModel.onTopicCreate(record); + + Assertions.assertEquals(topicName, clusterModel.topicName(topicId)); + + RemoveTopicRecord removeTopicRecord = new RemoveTopicRecord() + .setTopicId(topicId); + clusterModel.onTopicDelete(removeTopicRecord); + + Assertions.assertNull(clusterModel.topicName(topicId)); + } + + @Test + public void testCreatePartition() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + String topicName = "testTopic"; + Uuid topicId = Uuid.randomUuid(); + int partition = 0; + int brokerId = 1; + TopicPartition tp = new TopicPartition(topicName, partition); + + // create on non-exist broker + PartitionRecord partitionRecord = new PartitionRecord() + .setReplicas(List.of(brokerId, 2)) + .setTopicId(topicId) + .setPartitionId(partition); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertNull(clusterModel.replicaUpdater(brokerId, tp)); + + // create on non-exist topic + RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord() + .setBrokerId(brokerId); + clusterModel.onBrokerRegister(brokerRecord); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertNull(clusterModel.replicaUpdater(brokerId, tp)); + + // create with invalid replicas + TopicRecord topicRecord = new TopicRecord() + .setName(topicName) + .setTopicId(topicId); + clusterModel.onTopicCreate(topicRecord); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertNull(clusterModel.replicaUpdater(brokerId, tp)); + + partitionRecord.setReplicas(List.of(brokerId)); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertEquals(tp, clusterModel.replicaUpdater(brokerId, tp).topicPartition()); + } + + @Test + public void testChangePartition() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + String topicName = "testTopic"; + Uuid topicId = Uuid.randomUuid(); + int partition = 0; + int oldBrokerId = 1; + int newBrokerId = 2; + TopicPartition tp = new TopicPartition(topicName, partition); + + // reassign on non-exist broker + PartitionChangeRecord partitionChangeRecord = new PartitionChangeRecord() + .setReplicas(List.of(newBrokerId, 3)) + .setTopicId(topicId) + .setPartitionId(partition); + clusterModel.onPartitionChange(partitionChangeRecord); + Assertions.assertNull(clusterModel.replicaUpdater(newBrokerId, tp)); + + // create on non-exist topic + RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord() + .setBrokerId(newBrokerId); + clusterModel.onBrokerRegister(brokerRecord); + clusterModel.onPartitionChange(partitionChangeRecord); + Assertions.assertNull(clusterModel.replicaUpdater(newBrokerId, tp)); + + // create with invalid replicas + TopicRecord topicRecord = new TopicRecord() + .setName(topicName) + .setTopicId(topicId); + clusterModel.onTopicCreate(topicRecord); + clusterModel.onPartitionChange(partitionChangeRecord); + Assertions.assertNull(clusterModel.replicaUpdater(newBrokerId, tp)); + + RegisterBrokerRecord brokerRecord2 = new RegisterBrokerRecord() + .setBrokerId(oldBrokerId); + clusterModel.onBrokerRegister(brokerRecord2); + PartitionRecord partitionRecord = new PartitionRecord() + .setReplicas(List.of(oldBrokerId)) + .setTopicId(topicId) + .setPartitionId(partition); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertEquals(tp, clusterModel.replicaUpdater(oldBrokerId, tp).topicPartition()); + + partitionChangeRecord.setReplicas(List.of(newBrokerId)); + clusterModel.onPartitionChange(partitionChangeRecord); + Assertions.assertEquals(tp, clusterModel.replicaUpdater(newBrokerId, tp).topicPartition()); + Assertions.assertNull(clusterModel.replicaUpdater(oldBrokerId, tp)); + } + + @Test + public void testUpdateBroker() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + 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.updateBroker(brokerMetrics)); + + RegisterBrokerRecord registerBrokerRecord = new RegisterBrokerRecord() + .setBrokerId(brokerId); + clusterModel.onBrokerRegister(registerBrokerRecord); + Assertions.assertEquals(brokerId, clusterModel.brokerUpdater(brokerId).id()); + Assertions.assertTrue(clusterModel.updateBroker(brokerMetrics)); + } + + @Test + public void testUpdatePartition() { + ClusterModel clusterModel = new ClusterModel(new AutoBalancerControllerConfig(Collections.emptyMap(), false)); + String topicName = "testTopic"; + Uuid topicId = Uuid.randomUuid(); + int partition = 0; + int brokerId = 1; + + // update on non-exist topic + long now = System.currentTimeMillis(); + TopicPartitionMetrics topicPartitionMetrics = new TopicPartitionMetrics(now, brokerId, "", topicName, partition); + topicPartitionMetrics.put(RawMetricType.TOPIC_PARTITION_BYTES_IN, 10); + topicPartitionMetrics.put(RawMetricType.TOPIC_PARTITION_BYTES_OUT, 10); + topicPartitionMetrics.put(RawMetricType.PARTITION_SIZE, 10); + Assertions.assertFalse(clusterModel.updateTopicPartition(topicPartitionMetrics)); + + RegisterBrokerRecord registerBrokerRecord = new RegisterBrokerRecord() + .setBrokerId(brokerId); + clusterModel.onBrokerRegister(registerBrokerRecord); + TopicRecord topicRecord = new TopicRecord() + .setName(topicName) + .setTopicId(topicId); + clusterModel.onTopicCreate(topicRecord); + PartitionRecord partitionRecord = new PartitionRecord() + .setReplicas(List.of(brokerId)) + .setTopicId(topicId) + .setPartitionId(partition); + clusterModel.onPartitionCreate(partitionRecord); + Assertions.assertTrue(clusterModel.updateTopicPartition(topicPartitionMetrics)); + } + + @Test + public void testSnapshot() { + + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/AbstractLoadGenerator.java b/core/src/test/java/kafka/autobalancer/utils/AbstractLoadGenerator.java new file mode 100644 index 0000000000..18fef35704 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/AbstractLoadGenerator.java @@ -0,0 +1,70 @@ +/* + * 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.utils; + +import org.apache.kafka.common.utils.KafkaThread; + +import java.util.concurrent.atomic.AtomicInteger; + +public abstract class AbstractLoadGenerator implements Runnable { + protected final String topic; + protected final int partition; + protected static final int MSG_SIZE = 4096; // 4K + protected final AtomicInteger msgFailed; + protected final AtomicInteger msgSucceed; + protected boolean shutdown; + private final KafkaThread thread; + + public AbstractLoadGenerator(String topic, int partition) { + this.topic = topic; + this.partition = partition; + this.thread = KafkaThread.daemon("load-generator", this); + this.msgFailed = new AtomicInteger(0); + this.msgSucceed = new AtomicInteger(0); + } + + public void start() { + this.shutdown = false; + this.thread.start(); + } + + public void shutdown() { + this.shutdown = true; + try { + this.thread.join(); + } catch (InterruptedException ignored) { + + } + } + + public int msgFailed() { + return msgFailed.get(); + } + + public int msgSucceed() { + return msgSucceed.get(); + } + + public String getTopic() { + return topic; + } + + public int getPartition() { + return partition; + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/AutoBalancerClientsIntegrationTestHarness.java b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerClientsIntegrationTestHarness.java new file mode 100644 index 0000000000..e2ba321a60 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerClientsIntegrationTestHarness.java @@ -0,0 +1,170 @@ +/* + * 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.utils; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.CreateTopicsResult; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Assertions; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; + + +public abstract class AutoBalancerClientsIntegrationTestHarness extends AutoBalancerIntegrationTestHarness { + protected static final String TOPIC_0 = "TestTopic"; + protected static final String TOPIC_1 = "TestTopic1"; + protected static final String TOPIC_2 = "TestTopic2"; + protected static final String TOPIC_3 = "TestTopic3"; + protected static final String TOPIC_4 = "TestTopic4"; + + protected AdminClient adminClient; + + @Override + public void tearDown() { + this.adminClient.close(); + super.tearDown(); + } + + @Override + public void setUp() { + super.setUp(); + Properties adminProps = new Properties(); + adminProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + adminClient = AdminClient.create(adminProps); + try { + createTopic(TOPIC_0, 1, (short) 1); + } catch (Exception e) { + Assertions.fail("failed to create test topic"); + } + + // starting producer to verify that Kafka cluster is working fine + Properties producerProps = new Properties(); + producerProps.setProperty(ProducerConfig.ACKS_CONFIG, "-1"); + AtomicInteger producerFailed = new AtomicInteger(0); + try (Producer producer = createProducer(producerProps)) { + for (int i = 0; i < 10; i++) { + producer.send(new ProducerRecord<>(TOPIC_0, Integer.toString(i)), + (m, e) -> { + if (e != null) { + producerFailed.incrementAndGet(); + } + }); + } + } + Assertions.assertEquals(0, producerFailed.get()); + } + + protected boolean checkTopicCreation(String topicName, int numPartitions) { + try { + Map topics = adminClient.describeTopics(List.of(topicName)).allTopicNames().get(); + TopicDescription desc = topics.get(topicName); + if (desc == null) { + return false; + } + return desc.partitions().size() == numPartitions; + } catch (Exception e) { + return false; + } + } + + protected void createTopic(String topicName, int numPartitions, short replicaFactor) throws Exception { + NewTopic topic = new NewTopic(topicName, numPartitions, replicaFactor); + CreateTopicsResult createTopicsResult = adminClient.createTopics(Collections.singleton(topic)); + Assertions.assertNull(createTopicsResult.values().get(topicName).get()); + TestUtils.waitForCondition(() -> checkTopicCreation(topicName, numPartitions), 5000L, 1000L, + () -> "topic not ready in given time"); + } + + protected void createTopic(String topicName, Map> replicasAssignments) throws Exception { + NewTopic topic = new NewTopic(topicName, replicasAssignments); + CreateTopicsResult createTopicsResult = adminClient.createTopics(Collections.singleton(topic)); + Assertions.assertNull(createTopicsResult.values().get(topicName).get()); + TestUtils.waitForCondition(() -> checkTopicCreation(topicName, replicasAssignments.size()), 5000L, 1000L, + () -> "topic not ready in given time"); + } + + protected Producer createProducer(Properties overrides) { + Properties props = getProducerProperties(overrides); + KafkaProducer producer = new KafkaProducer<>(props); + Assertions.assertNotNull(producer); + return producer; + } + + protected Properties getProducerProperties(Properties overrides) { + Properties result = new Properties(); + + //populate defaults + result.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + result.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName()); + result.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName()); + + //apply overrides + if (overrides != null) { + result.putAll(overrides); + } + + return result; + } + + protected int getReplicaFor(String topic, int partition) { + try { + Map topics = adminClient.describeTopics(List.of(topic)).allTopicNames().get(); + TopicDescription desc = topics.get(topic); + if (desc == null) { + return -1; + } + Optional optionalTp = desc.partitions().stream().filter(p -> p.partition() == partition).findFirst(); + if (optionalTp.isEmpty()) { + return -1; + } + if (optionalTp.get().replicas().isEmpty()) { + return -1; + } + return optionalTp.get().replicas().get(0).id(); + } catch (Exception e) { + return -1; + } + } + + protected Optional getPartitionInfoFor(String topic, int partition) { + try { + Map topics = adminClient.describeTopics(List.of(topic)).allTopicNames().get(); + TopicDescription desc = topics.get(topic); + if (desc == null) { + return Optional.empty(); + } + return desc.partitions().stream().filter(p -> p.partition() == partition).findFirst(); + } catch (Exception e) { + return Optional.empty(); + } + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/AutoBalancerIntegrationTestHarness.java b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerIntegrationTestHarness.java new file mode 100644 index 0000000000..ccd2be12f4 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerIntegrationTestHarness.java @@ -0,0 +1,131 @@ +/* + * 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.utils; + +import kafka.autobalancer.config.AutoBalancerConfig; +import kafka.autobalancer.config.AutoBalancerControllerConfig; +import kafka.autobalancer.config.AutoBalancerMetricsReporterConfig; +import kafka.server.KafkaConfig; +import kafka.testkit.BrokerNode; +import kafka.testkit.ControllerNode; +import kafka.testkit.KafkaClusterTestKit; +import kafka.testkit.TestKitNodes; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.metadata.BrokerState; +import org.apache.kafka.test.TestUtils; + +import java.io.File; +import java.util.Collections; +import java.util.Map; + + +public abstract class AutoBalancerIntegrationTestHarness { + protected static final String METRIC_TOPIC = AutoBalancerConfig.DEFAULT_AUTO_BALANCER_TOPIC; + public static final String HOST = "localhost"; + + protected KafkaClusterTestKit cluster = null; + protected String bootstrapUrl; + + public void setUp() { + if (cluster != null) { + return; + } + + try { + // do not support multi-broker cluster since ElasticLogManager use shared singleton + TestKitNodes nodes = new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1).build(); + + int i = 0; + int[] port = AutoBalancerTestUtils.findLocalPorts(1); + for (BrokerNode broker : nodes.brokerNodes().values()) { + broker.propertyOverrides().put(KafkaConfig.ListenersProp(), "EXTERNAL://" + HOST + ":" + port[i]); + broker.propertyOverrides().put(AutoBalancerMetricsReporterConfig.config(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), HOST + ":" + port[i]); + broker.propertyOverrides().put(AutoBalancerMetricsReporterConfig.AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID, + AutoBalancerMetricsReporterConfig.DEFAULT_AUTO_BALANCER_METRICS_REPORTER_PRODUCER_CLIENT_ID + "-" + TestUtils.RANDOM.nextLong()); + broker.propertyOverrides().put(KafkaConfig.ElasticStreamEndpointProp(), "memory://"); + broker.propertyOverrides().put(KafkaConfig.ElasticStreamEnableProp(), "true"); + broker.propertyOverrides().putAll(overridingBrokerProps()); + broker.propertyOverrides().putAll(overridingNodeProps()); + i++; + } + for (ControllerNode controller : nodes.controllerNodes().values()) { + controller.propertyOverrides().putAll(overridingControllerProps()); + controller.propertyOverrides().putAll(overridingNodeProps()); + controller.propertyOverrides().put(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_ENABLE, "true"); + controller.propertyOverrides().put(KafkaConfig.ElasticStreamEnableProp(), "true"); + } + + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); + cluster = builder.build(); + cluster.format(); + cluster.startup(); + cluster.waitForReadyBrokers(); + + TestUtils.waitForCondition(() -> cluster.brokers().values().stream().noneMatch(b -> b.brokerState() != BrokerState.RUNNING), + TestUtils.DEFAULT_MAX_WAIT_MS, 1000, () -> "Broker never made it to RUNNING state."); + TestUtils.waitForCondition(() -> cluster.raftManagers().values().stream().noneMatch(r -> r.client().leaderAndEpoch().leaderId().isEmpty()), + TestUtils.DEFAULT_MAX_WAIT_MS, 1000, () -> "RaftManager was not initialized."); + } catch (Exception e) { + throw new RuntimeException("create cluster failed", e); + } + + bootstrapUrl = cluster.bootstrapServers(); + } + + public void tearDown() { + if (cluster != null) { + System.out.printf("tear down%n"); + try { + cluster.close(); + } catch (Exception e) { + throw new RuntimeException("close cluster failed", e); + } + } + } + + public String bootstrapServers() { + return bootstrapUrl; + } + + protected SecurityProtocol securityProtocol() { + return SecurityProtocol.PLAINTEXT; + } + + protected File trustStoreFile() { + return null; + } + + protected int clusterSize() { + return 1; + } + + protected Map overridingNodeProps() { + return Collections.emptyMap(); + } + + protected Map overridingBrokerProps() { + return Collections.emptyMap(); + } + + protected Map overridingControllerProps() { + return Collections.emptyMap(); + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/AutoBalancerTestUtils.java b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerTestUtils.java new file mode 100644 index 0000000000..fd3e3c6aed --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/AutoBalancerTestUtils.java @@ -0,0 +1,122 @@ +/* + * 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.utils; + +import org.apache.directory.api.util.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.net.ServerSocket; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + + +public final class AutoBalancerTestUtils { + private static final AtomicBoolean SHUTDOWN_HOOK_INSTALLED = new AtomicBoolean(false); + private static final Thread SHUTDOWN_HOOK; + private static final List FILES_TO_CLEAN_UP = Collections.synchronizedList(new ArrayList<>()); + + static { + SHUTDOWN_HOOK = new Thread(() -> { + Exception firstIssue = null; + for (File toCleanUp : FILES_TO_CLEAN_UP) { + if (!toCleanUp.exists()) { + continue; + } + try { + FileUtils.forceDelete(toCleanUp); + } catch (IOException issue) { + if (firstIssue == null) { + firstIssue = issue; + } else { + firstIssue.addSuppressed(issue); + } + } + } + if (firstIssue != null) { + System.err.println("unable to delete one or more files"); + firstIssue.printStackTrace(System.err); + throw new IllegalStateException(firstIssue); + } + }, "CCKafkaTestUtils cleanup hook"); + SHUTDOWN_HOOK.setUncaughtExceptionHandler((t, e) -> { + System.err.println("thread " + t.getName() + " died to uncaught exception"); + e.printStackTrace(System.err); + }); + } + + private AutoBalancerTestUtils() { + //utility class + } + + /** + * Cleanup the given file. + * + * @param toCleanUp File to cleanup. + * @return File to be cleaned up. + */ + public static File cleanup(File toCleanUp) { + if (SHUTDOWN_HOOK_INSTALLED.compareAndSet(false, true)) { + Runtime.getRuntime().addShutdownHook(SHUTDOWN_HOOK); + } + FILES_TO_CLEAN_UP.add(toCleanUp); + return toCleanUp; + } + + /** + * Find a local port. + * + * @return A local port to use. + */ + public static int[] findLocalPorts(int portNum) { + int[] ports = new int[portNum]; + List sockets = new ArrayList<>(); + for (int i = 0; i < portNum; i++) { + int port = -1; + while (port < 0) { + try { + ServerSocket socket = new ServerSocket(0); + socket.setReuseAddress(true); + port = socket.getLocalPort(); + ports[i] = port; + sockets.add(socket); + } catch (IOException ie) { + // let it go. + } + } + } + for (ServerSocket socket : sockets) { + try { + socket.close(); + } catch (IOException e) { + // Ignore IOException on close() + } + } + return ports; + } + + /** + * A functional interface for a task to run. + */ + @FunctionalInterface + public interface Task { + void run() throws Exception; + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/ConsumeLoadGenerator.java b/core/src/test/java/kafka/autobalancer/utils/ConsumeLoadGenerator.java new file mode 100644 index 0000000000..1a214b29fc --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/ConsumeLoadGenerator.java @@ -0,0 +1,58 @@ +/* + * 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.utils; + +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Assertions; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +public class ConsumeLoadGenerator extends AbstractLoadGenerator { + private final Properties props; + + public ConsumeLoadGenerator(Properties consumerProps, String topic, int partition) { + super(topic, partition); + this.props = consumerProps; + } + + @Override + public void run() { + try (final KafkaConsumer consumer = new KafkaConsumer<>(this.props)) { + List partitions = consumer.partitionsFor(topic); + Optional partitionInfo = partitions.stream().filter(p -> p.topic().equals(topic) && p.partition() == partition).findFirst(); + Assertions.assertTrue(partitionInfo.isPresent()); + List topicPartitions = List.of(new TopicPartition(partitionInfo.get().topic(), partitionInfo.get().partition())); + consumer.assign(topicPartitions); + consumer.seekToBeginning(topicPartitions); + while (!shutdown) { + try { + ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); + msgSucceed.addAndGet(records.count()); + } catch (Exception ignored) { + + } + } + } + } +} diff --git a/core/src/test/java/kafka/autobalancer/utils/ProduceLoadGenerator.java b/core/src/test/java/kafka/autobalancer/utils/ProduceLoadGenerator.java new file mode 100644 index 0000000000..18740b5f62 --- /dev/null +++ b/core/src/test/java/kafka/autobalancer/utils/ProduceLoadGenerator.java @@ -0,0 +1,98 @@ +/* + * 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.utils; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.test.TestUtils; + +import java.util.Properties; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class ProduceLoadGenerator extends AbstractLoadGenerator { + private final Properties props; + private final byte[] payload; + private final Lock lock = new ReentrantLock(); + private final Condition cond = lock.newCondition(); + private final ScheduledExecutorService executorService; + private final int throughput; + private int token; + + public ProduceLoadGenerator(Properties producerProps, String topic, int partition, int load) { + super(topic, partition); + this.props = producerProps; + this.throughput = load * 1024 / MSG_SIZE; + this.payload = new byte[MSG_SIZE]; + generatePayload(payload); + this.token = throughput; + this.executorService = Executors.newSingleThreadScheduledExecutor(); + } + + private void generatePayload(byte[] payload) { + for (int j = 0; j < payload.length; ++j) + payload[j] = (byte) (TestUtils.RANDOM.nextInt(26) + 65); + } + + @Override + public void start() { + super.start(); + this.executorService.scheduleAtFixedRate(() -> { + lock.lock(); + try { + this.token = throughput; + cond.signal(); + } finally { + lock.unlock(); + } + }, 0, 1, TimeUnit.SECONDS); + } + + @Override + public void run() { + try (final KafkaProducer producer = new KafkaProducer<>(this.props)) { + while (!shutdown) { + lock.lock(); + try { + while (this.token == 0) { + cond.await(); + } + this.token--; + } catch (InterruptedException ignored) { + + } finally { + lock.unlock(); + } + try { + producer.send(new ProducerRecord<>(topic, partition, null, payload), (m, e) -> { + if (e != null) { + msgFailed.incrementAndGet(); + } + }); + msgSucceed.incrementAndGet(); + } catch (Exception e) { + msgFailed.incrementAndGet(); + } + } + } + } +}