From 152ea3a0ced3d2c35c1b0baab3aeba2629bf5261 Mon Sep 17 00:00:00 2001 From: Yennick Trevels Date: Tue, 27 Jun 2017 21:43:10 +0200 Subject: [PATCH 001/192] Docker build hardening (#79) Don't add non-existing folders to docker image and handle signals in start script to have a clean container shutdown. --- docker/Dockerfile | 5 +---- docker/kafka-monitor-docker-entry.sh | 8 +++++++- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index fcd49e39..c47fbffb 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -17,9 +17,6 @@ MAINTAINER coffeepac@gmail.com WORKDIR /opt/kafka-monitor ADD build/ build/ -ADD core/build/ core/build/ -ADD services/build/ services/build/ -ADD tests/build/ tests/build/ ADD bin/kafka-monitor-start.sh bin/kafka-monitor-start.sh ADD bin/kmf-run-class.sh bin/kmf-run-class.sh ADD config/kafka-monitor.properties config/kafka-monitor.properties @@ -27,4 +24,4 @@ ADD config/log4j.properties config/log4j.properties ADD docker/kafka-monitor-docker-entry.sh kafka-monitor-docker-entry.sh ADD webapp/ webapp/ -CMD ["/opt/kafka-monitor/kafka-monitor-docker-entry.sh"] \ No newline at end of file +CMD ["/opt/kafka-monitor/kafka-monitor-docker-entry.sh"] diff --git a/docker/kafka-monitor-docker-entry.sh b/docker/kafka-monitor-docker-entry.sh index 3abc003b..946b6e66 100755 --- a/docker/kafka-monitor-docker-entry.sh +++ b/docker/kafka-monitor-docker-entry.sh @@ -15,7 +15,13 @@ set -x +# SIGTERM-handler +trap 'pkill java; exit 130' SIGINT +trap 'pkill java; exit 143' SIGTERM + # wait for DNS services to be available sleep 10 -bin/kafka-monitor-start.sh config/kafka-monitor.properties +bin/kafka-monitor-start.sh config/kafka-monitor.properties & + +wait $! \ No newline at end of file From c4915d9427b9b7c1b2f9318de280794f66ad6534 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 29 Jun 2017 09:58:27 -0700 Subject: [PATCH 002/192] Keep properties of the monitor topic updated (#83) --- config/kafka-monitor.properties | 2 ++ .../MultiClusterTopicManagementService.java | 23 ++++++++++++++++--- .../linkedin/kmf/services/ProduceService.java | 2 +- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 91779669..b4a66a71 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -53,6 +53,8 @@ "topic-management.rebalance.interval.ms" : 600000, "topic-management.topicFactory.props": { }, + "topic-management.topic.props": { + }, "produce.producer.props": { "client.id": "kmf-client-id" }, diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index d869d41c..2994029f 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -36,6 +36,8 @@ import kafka.admin.RackAwareMode; import kafka.cluster.Broker; import kafka.common.TopicAndPartition; +import kafka.server.ConfigType; +import kafka.server.KafkaConfig; import kafka.utils.ZkUtils; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.apache.kafka.common.Node; @@ -193,9 +195,16 @@ static class TopicManagementHelper { _minPartitionsToBrokersRatio = config.getDouble(TopicManagementServiceConfig.PARTITIONS_TO_BROKERS_RATIO_CONFIG); _minPartitionNum = config.getInt(TopicManagementServiceConfig.MIN_PARTITION_NUM_CONFIG); String topicFactoryClassName = config.getString(TopicManagementServiceConfig.TOPIC_FACTORY_CLASS_CONFIG); + _topicProperties = new Properties(); - if (props.containsKey(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)) - _topicProperties.putAll((Map) props.get(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)); + if (props.containsKey(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)) { + for (Map.Entry entry: ((Map) props.get(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)).entrySet()) + _topicProperties.put(entry.getKey(), entry.getValue().toString()); + } + if (!_topicProperties.containsKey(KafkaConfig.MinInSyncReplicasProp())) { + int defaultMinIsr = Math.max(_replicationFactor - 1, 1); + _topicProperties.setProperty(KafkaConfig.MinInSyncReplicasProp(), Integer.toString(defaultMinIsr)); + } Map topicFactoryConfig = props.containsKey(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) ? (Map) props.get(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) : new HashMap(); @@ -245,10 +254,18 @@ void maybeReassignPartitionAndElectLeader() throws Exception { _replicationFactor, currentReplicationFactor, _topic, _zkConnect)); if (_replicationFactor > currentReplicationFactor && zkUtils.getPartitionsBeingReassigned().isEmpty()) { - LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}", _topic, _zkConnect); + LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}" + + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, _replicationFactor); reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), _replicationFactor); } + Properties currentProperties = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic(), _topic); + if (!currentProperties.equals(_topicProperties)) { + LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + + "in cluster {} from {} to {}.", _topic, _zkConnect, currentProperties, _topicProperties); + AdminUtils.changeTopicConfig(zkUtils, _topic, _topicProperties); + } + if (partitionInfoList.size() >= brokers.size() && someBrokerNotPreferredLeader(partitionInfoList, brokers) && zkUtils.getPartitionsBeingReassigned().isEmpty()) { diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 8e2e1290..f9886cf3 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -130,7 +130,7 @@ private void initializeProducer() throws Exception { // Assign default config. This has the lowest priority. producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000"); - producerProps.put(ProducerConfig.RETRIES_CONFIG, 3); + producerProps.put(ProducerConfig.RETRIES_CONFIG, "3"); producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true"); producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); From 4c7ed7c1001f4c897cf4fa58cc09e31213ac2059 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 11 Jul 2017 14:39:45 -0700 Subject: [PATCH 003/192] Override min.insync.isr for the monitor topic only if it is specified by the user (#85) --- .../java/com/linkedin/kmf/common/Utils.java | 7 ------- .../MultiClusterTopicManagementService.java | 18 ++++++++++-------- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index e1c8494b..59ac0396 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -89,16 +89,9 @@ public static int createMonitoringTopicIfNotExists(String zkUrl, String topic, i if (AdminUtils.topicExists(zkUtils, topic)) { return getPartitionNumForTopic(zkUrl, topic); } - int brokerCount = zkUtils.getAllBrokersInCluster().size(); - int partitionCount = (int) Math.ceil(brokerCount * partitionToBrokerRatio); - int defaultMinIsr = Math.max(replicationFactor - 1, 1); - if (!topicConfig.containsKey(KafkaConfig.MinInSyncReplicasProp())) { - topicConfig.setProperty(KafkaConfig.MinInSyncReplicasProp(), Integer.toString(defaultMinIsr)); - } - try { AdminUtils.createTopic(zkUtils, topic, partitionCount, replicationFactor, topicConfig, RackAwareMode.Enforced$.MODULE$); } catch (TopicExistsException e) { diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 2994029f..31501c42 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -37,7 +37,6 @@ import kafka.cluster.Broker; import kafka.common.TopicAndPartition; import kafka.server.ConfigType; -import kafka.server.KafkaConfig; import kafka.utils.ZkUtils; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.apache.kafka.common.Node; @@ -201,10 +200,6 @@ static class TopicManagementHelper { for (Map.Entry entry: ((Map) props.get(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)).entrySet()) _topicProperties.put(entry.getKey(), entry.getValue().toString()); } - if (!_topicProperties.containsKey(KafkaConfig.MinInSyncReplicasProp())) { - int defaultMinIsr = Math.max(_replicationFactor - 1, 1); - _topicProperties.setProperty(KafkaConfig.MinInSyncReplicasProp(), Integer.toString(defaultMinIsr)); - } Map topicFactoryConfig = props.containsKey(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) ? (Map) props.get(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) : new HashMap(); @@ -259,11 +254,18 @@ void maybeReassignPartitionAndElectLeader() throws Exception { reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), _replicationFactor); } + // Update the properties of the monitor topic if any config is different from the user-specified config Properties currentProperties = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic(), _topic); - if (!currentProperties.equals(_topicProperties)) { + Properties expectedProperties = new Properties(); + for (Object key: currentProperties.keySet()) + expectedProperties.put(key, currentProperties.get(key)); + for (Object key: _topicProperties.keySet()) + expectedProperties.put(key, _topicProperties.get(key)); + + if (!currentProperties.equals(expectedProperties)) { LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " - + "in cluster {} from {} to {}.", _topic, _zkConnect, currentProperties, _topicProperties); - AdminUtils.changeTopicConfig(zkUtils, _topic, _topicProperties); + + "in cluster {} from {} to {}.", _topic, _zkConnect, currentProperties, expectedProperties); + AdminUtils.changeTopicConfig(zkUtils, _topic, expectedProperties); } if (partitionInfoList.size() >= brokers.size() && From 312abe0b0b32c94f4f54e302e546ab217a5f785b Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 26 Jul 2017 17:35:07 -0700 Subject: [PATCH 004/192] Use current replication factor to expand partition if it is larger than the configured value (#87) --- src/main/java/com/linkedin/kmf/KafkaMonitor.java | 14 +++++++++----- .../MultiClusterTopicManagementService.java | 14 +++++++------- .../configs/TopicManagementServiceConfig.java | 5 +++-- 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index daa623c4..e2f93d99 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -118,18 +118,22 @@ public void run() { private void checkHealth() { for (Map.Entry entry: _apps.entrySet()) { - if (!entry.getValue().isRunning()) { + if (!entry.getValue().isRunning()) _offlineRunnables.putIfAbsent(entry.getKey(), entry.getValue()); - LOG.error("App " + entry.getKey() + " is not fully running."); - } } for (Map.Entry entry: _services.entrySet()) { - if (!entry.getValue().isRunning()) { + if (!entry.getValue().isRunning()) _offlineRunnables.putIfAbsent(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry: _offlineRunnables.entrySet()) { + if (entry.getValue() instanceof App) + LOG.error("App " + entry.getKey() + " is not fully running."); + else LOG.error("Service " + entry.getKey() + " is not fully running."); - } } + } public synchronized void stop() { diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 31501c42..10005bf4 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -242,16 +242,16 @@ void maybeReassignPartitionAndElectLeader() throws Exception { throw new IllegalStateException("Topic " + _topic + " does not exist in cluster " + _zkConnect); int currentReplicationFactor = getReplicationFactor(partitionInfoList); + int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); if (_replicationFactor < currentReplicationFactor) - throw new RuntimeException(String.format("Configured replication factor %d " - + "is smaller than the current replication factor %d of the topic %s in cluster %s", - _replicationFactor, currentReplicationFactor, _topic, _zkConnect)); + LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster {}", + _replicationFactor, currentReplicationFactor, _topic, _zkConnect); - if (_replicationFactor > currentReplicationFactor && zkUtils.getPartitionsBeingReassigned().isEmpty()) { + if (expectedReplicationFactor > currentReplicationFactor && zkUtils.getPartitionsBeingReassigned().isEmpty()) { LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}" - + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, _replicationFactor); - reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), _replicationFactor); + + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, expectedReplicationFactor); + reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); } // Update the properties of the monitor topic if any config is different from the user-specified config @@ -272,7 +272,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { someBrokerNotPreferredLeader(partitionInfoList, brokers) && zkUtils.getPartitionsBeingReassigned().isEmpty()) { LOG.info("MultiClusterTopicManagementService will reassign partitions of the topic {} in cluster {}", _topic, _zkConnect); - reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), _replicationFactor); + reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); } if (partitionInfoList.size() >= brokers.size() && diff --git a/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java index 36a1b947..9f79b30f 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java @@ -40,8 +40,9 @@ public class TopicManagementServiceConfig extends AbstractConfig { + " This config provides a loose lower bound on the partition number of the monitor topic when the topic is created or when partition is added."; public static final String TOPIC_REPLICATION_FACTOR_CONFIG = "topic-management.replicationFactor"; - public static final String TOPIC_REPLICATION_FACTOR_DOC = "When a topic is created automatically this is the " - + "replication factor used."; + public static final String TOPIC_REPLICATION_FACTOR_DOC = "This replication factor is used to create the monitor topic. " + + "The larger one of the current replication factor and the configured replication factor is used to expand partition " + + "of the monitor topic."; public static final String TOPIC_CREATION_ENABLED_CONFIG = "topic-management.topicCreationEnabled"; public static final String TOPIC_CREATION_ENABLED_DOC = String.format("When true this service automatically creates the topic named" From 1f1228f6ca9ced9424237cc244ee328ba28852d1 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 7 Aug 2017 13:56:51 -0700 Subject: [PATCH 005/192] Upgrade kafka dependency to 0.10.2.1 (#92) --- README.md | 13 +++++++++++++ build.gradle | 4 ++-- config/multi-cluster-monitor.properties | 13 +++++++++---- .../kmf/services/TopicManagementServiceTest.java | 3 ++- 4 files changed, 26 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index 7b5ba295..33c94413 100644 --- a/README.md +++ b/README.md @@ -25,6 +25,11 @@ the monitor topic. Kafka Monitor requires Gradle 2.0 or higher. Java 7 should be used for building in order to support both Java 7 and Java 8 at runtime. +Kafka Monitor supports Apache Kafka 0.8 to 0.10: +- Use branch 0.8.2.2 to work with Apache Kafka 0.8 +- Use branch 0.9.0.1 to work with Apache Kafka 0.9 +- Use master branch to work with Apache Kafka 0.10.2.1 + Kafka Monitor supports Apache Kafka 0.8 and 0.9. Use branch 0.8.2.2 to monitor Apache Kafka cluster 0.8. Use branch 0.9.0.1 to compile with Kafka 0.9. Use master branch to compile with Kafka 0.10. @@ -77,6 +82,14 @@ $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties $ ./bin/single-cluster-monitor.sh --topic test --broker-list localhost:9092 --zookeeper localhost:2181 ``` +### Run MultiClusterMonitor app to monitor a pipeline of Kafka clusters +Edit `config/multi-cluster-monitor.properties` to specify the right broker and +zookeeper url as suggested by the comment in the properties file + +``` +$ ./bin/kafka-monitor-start.sh config/multi-cluster-monitor.properties +``` + ### Get metric values (e.g. service availability, message loss rate) in real-time as time series graphs Open ```localhost:8000/index.html``` in your web browser diff --git a/build.gradle b/build.gradle index 9db41aca..a00873d6 100644 --- a/build.gradle +++ b/build.gradle @@ -16,8 +16,8 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.11:0.10.1.1' - compile 'org.apache.kafka:kafka-clients:0.10.1.1' + compile 'org.apache.kafka:kafka_2.11:0.10.2.1' + compile 'org.apache.kafka:kafka-clients:0.10.2.1' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' diff --git a/config/multi-cluster-monitor.properties b/config/multi-cluster-monitor.properties index 6c35ec19..a236503d 100644 --- a/config/multi-cluster-monitor.properties +++ b/config/multi-cluster-monitor.properties @@ -17,8 +17,9 @@ "class.name": "com.linkedin.kmf.apps.MultiClusterMonitor", "topic": "kafka-monitor-topic", + # Configure Produce Service to produce to the first cluster of the pipeline "produce.service.props": { - "zookeeper.connect": "localhost:2181/cluster1", + "zookeeper.connect": "localhost:2181/first_cluster", "bootstrap.servers": "localhost:9092", "produce.record.delay.ms": 100, "produce.producer.props": { @@ -26,8 +27,9 @@ } }, + # Configure Consume Service to consume from the last cluster of the pipeline "consume.service.props": { - "zookeeper.connect": "localhost:2181/cluster2", + "zookeeper.connect": "localhost:2181/last_cluster", "bootstrap.servers": "localhost:9095", "consume.latency.sla.ms": "20000", "consume.consumer.props": { @@ -37,7 +39,7 @@ "topic.management.props.per.cluster" : { "first-cluster" : { - "zookeeper.connect": "localhost:2181/cluster1", + "zookeeper.connect": "localhost:2181/first_cluster", "topic-management.topicCreationEnabled": true, "topic-management.replicationFactor" : 1, "topic-management.partitionsToBrokersRatio" : 2.0, @@ -45,9 +47,12 @@ "topic-management.topicFactory.props": { } }, + + # If there are more than two clusters in the pipeline, + # add one property map for each one of them. "last-cluster" : { - "zookeeper.connect": "localhost:2181/cluster2", + "zookeeper.connect": "localhost:2181/last_cluster", "topic-management.topicCreationEnabled": true, "topic-management.replicationFactor" : 1, "topic-management.partitionsToBrokersRatio" : 2.0, diff --git a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java index 2e997c24..97f6d5c7 100644 --- a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java @@ -14,6 +14,7 @@ import kafka.cluster.Broker; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.testng.Assert; import org.testng.annotations.Test; import com.linkedin.kmf.services.MultiClusterTopicManagementService.TopicManagementHelper; @@ -26,7 +27,7 @@ public class TopicManagementServiceTest { private List brokers(int brokerCount) { List brokers = new ArrayList<>(); for (int i = 0; i < brokerCount; i++) { - brokers.add(new Broker(i, "", -1, null)); + brokers.add(new Broker(i, "", -1, null, SecurityProtocol.PLAINTEXT)); } return brokers; } From 8f6a2732d46f48ed237ae0b28b303ed430c3255f Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 10 Aug 2017 13:01:33 -0700 Subject: [PATCH 006/192] Reassign partitions with the user-configured replication factor if the replication factor of partitions of the monitor topic is inconsistent (#94) --- .../MultiClusterTopicManagementService.java | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 10005bf4..ae77fdd2 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -298,10 +298,19 @@ private static void reassignPartitions(ZkUtils zkUtils, Collection broke for (Broker broker : brokers) { brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), broker.rack())); } - scala.collection.Map> partitionToReplicas = + scala.collection.Map> newAssignment = AdminUtils.assignReplicasToBrokers(brokersMetadata, partitionCount, replicationFactor, 0, 0); - String jsonReassignmentData = formatAsReassignmentJson(topic, partitionToReplicas); - zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath(), jsonReassignmentData, zkUtils.DefaultAcls()); + + scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); + topicList.$plus$eq(topic); + scala.collection.Map> currentAssignment = zkUtils.getPartitionAssignmentForTopics(topicList).apply(topic); + String currentAssignmentJson = formatAsReassignmentJson(topic, currentAssignment); + String newAssignmentJson = formatAsReassignmentJson(topic, newAssignment); + + LOG.info("Reassign partitions for topic " + topic); + LOG.info("Current partition replica assignment " + currentAssignmentJson); + LOG.info("New partition replica assignment " + newAssignmentJson); + zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath(), newAssignmentJson, zkUtils.DefaultAcls()); } private static List getPartitionInfo(ZkUtils zkUtils, String topic) { @@ -335,7 +344,8 @@ static int getReplicationFactor(List partitionInfoList) { for (PartitionInfo partitionInfo : partitionInfoList) { if (replicationFactor != partitionInfo.replicas().length) { String topic = partitionInfoList.get(0).topic(); - throw new RuntimeException("Partitions of the topic " + topic + " have different replication factor"); + LOG.warn("Partitions of the topic " + topic + " have different replication factor"); + return -1; } } return replicationFactor; From 5472679b73fa8a8911bb3a940abb186f1ccd2441 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 11 Aug 2017 14:56:30 -0700 Subject: [PATCH 007/192] ProduceAvailability should be 1.0 as long as ProduceService does not see exception (#95) --- src/main/java/com/linkedin/kmf/services/ProduceService.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index f9886cf3..94e65e8d 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -250,6 +250,11 @@ public double measure(MetricConfig config, long now) { // If there is either succeeded or failed produce to a partition, consider its availability as 0. if (recordsProduced + produceError > 0) { availabilitySum += recordsProduced / (recordsProduced + produceError); + } else { + // A partition's availability is 1.0 as long as there is no error when kafka-monitor produces to this partition + // In the case that it takes too long to produce this message, we consider this to be a performance issue + // and it will be captured by the ConsumeAvailability if the latency exceeds consume.latency.sla.ms. + availabilitySum += 1.0; } } // Assign equal weight to per-partition availability when calculating overall availability From 0c30b1cc2ebfef8ba171288008bc935c0fffd693 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 24 Aug 2017 14:57:27 -0700 Subject: [PATCH 008/192] Add a debug level log statment in ProduceService --- src/main/java/com/linkedin/kmf/services/ProduceService.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 94e65e8d..b2d2fbb3 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -320,6 +320,8 @@ public void run() { private class NewPartitionHandler implements Runnable { public void run() { + LOG.debug("{}/ProduceService check partition number for topic {}.", _name, _topic); + int currentPartitionCount = Utils.getPartitionNumForTopic(_zkConnect, _topic); if (currentPartitionCount <= 0) { LOG.info("{}/ProduceService topic {} does not exist.", _name, _topic); From a7a9b3f77221459aa74b061637e92038702c3a10 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 25 Aug 2017 21:47:57 -0700 Subject: [PATCH 009/192] Minor change to README --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 33c94413..788aebfc 100644 --- a/README.md +++ b/README.md @@ -82,7 +82,7 @@ $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties $ ./bin/single-cluster-monitor.sh --topic test --broker-list localhost:9092 --zookeeper localhost:2181 ``` -### Run MultiClusterMonitor app to monitor a pipeline of Kafka clusters +### Run MultiClusterMonitor app to monitor a pipeline of Kafka clusters connected by MirrorMaker Edit `config/multi-cluster-monitor.properties` to specify the right broker and zookeeper url as suggested by the comment in the properties file From 1a116fd1b7b7ae3fac128a1539424e81d4df45ac Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 30 Aug 2017 17:11:26 -0700 Subject: [PATCH 010/192] Fix NPE in ProduceService by incrementing partitionNum only after we have created metrics for all partitions (#98) --- .../linkedin/kmf/services/ProduceService.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index b2d2fbb3..43eeb3f7 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -101,8 +101,6 @@ public ProduceService(Map props, String name) throws Exception { } } - _partitionNum.set(Utils.getPartitionNumForTopic(_zkConnect, _topic)); - if (producerClass.equals(NewProducer.class.getCanonicalName()) || producerClass.equals(NewProducer.class.getSimpleName())) { _producerClassName = NewProducer.class.getCanonicalName(); } else { @@ -148,15 +146,15 @@ private void initializeProducer() throws Exception { @Override public synchronized void start() { if (_running.compareAndSet(false, true)) { - initializeStateForPartitions(); + int partitionNum = Utils.getPartitionNumForTopic(_zkConnect, _topic); + initializeStateForPartitions(partitionNum); _handleNewPartitionsExecutor.scheduleWithFixedDelay(new NewPartitionHandler(), 1000, 30000, TimeUnit.MILLISECONDS); LOG.info("{}/ProduceService started", _name); } } - private void initializeStateForPartitions() { - Map keyMapping = generateKeyMappings(); - int partitionNum = _partitionNum.get(); + private void initializeStateForPartitions(int partitionNum) { + Map keyMapping = generateKeyMappings(partitionNum); for (int partition = 0; partition < partitionNum; partition++) { String key = keyMapping.get(partition); //This is what preserves sequence numbers across restarts @@ -165,11 +163,11 @@ private void initializeStateForPartitions() { _sensors.addPartitionSensors(partition); } _produceExecutor.scheduleWithFixedDelay(new ProduceRunnable(partition, key), _produceDelayMs, _produceDelayMs, TimeUnit.MILLISECONDS); + _partitionNum.set(partitionNum); } } - private Map generateKeyMappings() { - int partitionNum = _partitionNum.get(); + private Map generateKeyMappings(int partitionNum) { HashMap keyMapping = new HashMap<>(); int nextInt = 0; @@ -322,11 +320,11 @@ private class NewPartitionHandler implements Runnable { public void run() { LOG.debug("{}/ProduceService check partition number for topic {}.", _name, _topic); - int currentPartitionCount = Utils.getPartitionNumForTopic(_zkConnect, _topic); - if (currentPartitionCount <= 0) { + int currentPartitionNum = Utils.getPartitionNumForTopic(_zkConnect, _topic); + if (currentPartitionNum <= 0) { LOG.info("{}/ProduceService topic {} does not exist.", _name, _topic); return; - } else if (currentPartitionCount == _partitionNum.get()) { + } else if (currentPartitionNum == _partitionNum.get()) { return; } LOG.info("{}/ProduceService detected new partitions of topic {}", _name, _topic); @@ -338,7 +336,6 @@ public void run() { throw new IllegalStateException(e); } _producer.close(); - _partitionNum.set(currentPartitionCount); try { initializeProducer(); } catch (Exception e) { @@ -346,7 +343,7 @@ public void run() { throw new IllegalStateException(e); } _produceExecutor = Executors.newScheduledThreadPool(_threadsNum); - initializeStateForPartitions(); + initializeStateForPartitions(currentPartitionNum); LOG.info("New partitions added to monitoring."); } } From c0fbee30b50268c73f2bb5b4942b941529574694 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 30 Aug 2017 17:14:24 -0700 Subject: [PATCH 011/192] Add config produce.treat.zero.throughput.as.unavailable with default value set to true (#97) --- .../com/linkedin/kmf/services/ProduceService.java | 13 +++++++++---- .../kmf/services/configs/ProduceServiceConfig.java | 11 +++++++++++ 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 43eeb3f7..9902435b 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -74,6 +74,7 @@ public class ProduceService implements Service { private final String _producerClassName; private final int _threadsNum; private final String _zkConnect; + private final boolean _treatZeroThroughputAsUnavailable; public ProduceService(Map props, String name) throws Exception { _name = name; @@ -89,6 +90,7 @@ public ProduceService(Map props, String name) throws Exception { _produceDelayMs = config.getInt(ProduceServiceConfig.PRODUCE_RECORD_DELAY_MS_CONFIG); _recordSize = config.getInt(ProduceServiceConfig.PRODUCE_RECORD_SIZE_BYTE_CONFIG); _sync = config.getBoolean(ProduceServiceConfig.PRODUCE_SYNC_CONFIG); + _treatZeroThroughputAsUnavailable = config.getBoolean(ProduceServiceConfig.PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_CONFIG); _partitionNum = new AtomicInteger(0); _running = new AtomicBoolean(false); _nextIndexPerPartition = new ConcurrentHashMap<>(); @@ -248,10 +250,13 @@ public double measure(MetricConfig config, long now) { // If there is either succeeded or failed produce to a partition, consider its availability as 0. if (recordsProduced + produceError > 0) { availabilitySum += recordsProduced / (recordsProduced + produceError); - } else { - // A partition's availability is 1.0 as long as there is no error when kafka-monitor produces to this partition - // In the case that it takes too long to produce this message, we consider this to be a performance issue - // and it will be captured by the ConsumeAvailability if the latency exceeds consume.latency.sla.ms. + } else if (!_treatZeroThroughputAsUnavailable) { + // If user configures treatZeroThroughputAsUnavailable to be false, a partition's availability + // is 1.0 as long as there is no exception thrown from producer. + // This allows kafka admin to exactly monitor the availability experienced by Kafka users which + // will block and retry for a certain amount of time based on its configuration (e.g. retries, retry.backoff.ms). + // Note that if it takes a long time for messages to be retries and sent, the latency in the ConsumeService + // will increase and it will reduce ConsumeAvailability if the latency exceeds consume.latency.sla.ms availabilitySum += 1.0; } } diff --git a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java index 0a613971..b10894a8 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java @@ -55,6 +55,12 @@ public class ProduceServiceConfig extends AbstractConfig { public static final String PRODUCER_PROPS_CONFIG = "produce.producer.props"; public static final String PRODUCER_PROPS_DOC = "The properties used to config producer in produce service."; + public static final String PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_CONFIG = "produce.treat.zero.throughput.as.unavailable"; + public static final String PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_DOC = "If it is set to true, produce availability is set to 0 " + + "if no message can be produced, regardless of whether there is exception. If this is set to false, availability will only drop below 1 if there is exception " + + "thrown from producer. Depending on the producer configuration, it may take a few minutes for producer to be blocked before it throws exception. Advanced user " + + "may want to set this flag to false to exactly measure the availability experienced by users"; + static { CONFIG = new ConfigDef().define(ZOOKEEPER_CONNECT_CONFIG, ConfigDef.Type.STRING, @@ -98,6 +104,11 @@ public class ProduceServiceConfig extends AbstractConfig { 100, ConfigDef.Importance.LOW, PRODUCE_RECORD_SIZE_BYTE_DOC) + .define(PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_CONFIG, + ConfigDef.Type.BOOLEAN, + true, + ConfigDef.Importance.MEDIUM, + PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_DOC) .define(PRODUCE_THREAD_NUM_CONFIG, ConfigDef.Type.INT, 5, From d442f47ee0d0232e40bec55151fe7f8569228693 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 5 Sep 2017 10:00:53 -0700 Subject: [PATCH 012/192] Add instructions for running SASL enabled client in the README --- README.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/README.md b/README.md index 788aebfc..0483c905 100644 --- a/README.md +++ b/README.md @@ -77,6 +77,15 @@ $ ./gradlew jar $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties ``` +### Run Kafka Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) +Edit `config/kafka-monitor.properties` to specify custom configurations for prodcuer in the key/value map `produce.producer.props` in +`config/kafka-monitor.properties`. Similarly specify configurations for +consumer as well. The documentation for producer and consumer in the key/value maps can be found in the Apache Kafka wiki. + +``` +$ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties +``` + ### Run SingleClusterMonitor app to monitor kafka cluster ``` $ ./bin/single-cluster-monitor.sh --topic test --broker-list localhost:9092 --zookeeper localhost:2181 From fa9377adb596725c761da1b7ab710a95c5c8b915 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 17 Nov 2017 13:56:30 -0800 Subject: [PATCH 013/192] Set retention.ms of the monitor topic to 1 hour in the example kafka-monitor.properties --- config/kafka-monitor.properties | 1 + 1 file changed, 1 insertion(+) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index b4a66a71..60f36ac3 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -54,6 +54,7 @@ "topic-management.topicFactory.props": { }, "topic-management.topic.props": { + "retention.ms": "3600000" }, "produce.producer.props": { "client.id": "kmf-client-id" From 6722470d21174ee62bea1cda81dfb0a52ed40697 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 21 Nov 2017 10:10:19 -0800 Subject: [PATCH 014/192] Specify produce availability and consume availability in the README --- README.md | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 0483c905..5deceb80 100644 --- a/README.md +++ b/README.md @@ -87,6 +87,11 @@ $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties ``` ### Run SingleClusterMonitor app to monitor kafka cluster + +Metrics `produce-availability-avg` and `consume-availability-avg` demonstrate +whether messages can be properly produced to and consumed from this cluster. +See Service Overview wiki for how these metrics are derived. + ``` $ ./bin/single-cluster-monitor.sh --topic test --broker-list localhost:9092 --zookeeper localhost:2181 ``` @@ -95,6 +100,11 @@ $ ./bin/single-cluster-monitor.sh --topic test --broker-list localhost:9092 --zo Edit `config/multi-cluster-monitor.properties` to specify the right broker and zookeeper url as suggested by the comment in the properties file +Metrics `produce-availability-avg` and `consume-availability-avg` demonstrate +whether messages can be properly produced to the source cluster and consumed +from the destination cluster. See config/multi-cluster-monitor.properties for +the full jmx path for these metrics. + ``` $ ./bin/kafka-monitor-start.sh config/multi-cluster-monitor.properties ``` @@ -104,9 +114,11 @@ Open ```localhost:8000/index.html``` in your web browser You can edit webapp/index.html to easily add new metrics to be displayed. -### Query metric value (e.g. service availability) via HTTP request +### Query metric value (e.g. produce availability and consume availability) via HTTP request ``` curl localhost:8778/jolokia/read/kmf.services:type=produce-service,name=*/produce-availability-avg + +curl localhost:8778/jolokia/read/kmf.services:type=consume-service,name=*/consume-availability-avg ``` You can query other JMX metric value as well by substituting object-name and From 34110d6509bcc2b6f1b830a41a9c69ea48fdd8fb Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 12 Jan 2018 10:05:58 -0800 Subject: [PATCH 015/192] Fix travis configuration --- .travis.yml | 2 -- src/main/java/com/linkedin/kmf/services/ProduceService.java | 4 ++-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 0d5889e9..45e9d6b5 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,6 +1,4 @@ language: java jdk: - - oraclejdk7 - - openjdk7 - oraclejdk8 diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 9902435b..0f0f481d 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -165,8 +165,8 @@ private void initializeStateForPartitions(int partitionNum) { _sensors.addPartitionSensors(partition); } _produceExecutor.scheduleWithFixedDelay(new ProduceRunnable(partition, key), _produceDelayMs, _produceDelayMs, TimeUnit.MILLISECONDS); - _partitionNum.set(partitionNum); } + _partitionNum.set(partitionNum); } private Map generateKeyMappings(int partitionNum) { @@ -367,4 +367,4 @@ public Thread newThread(Runnable r) { } } -} \ No newline at end of file +} From 7df33a2b7b9dfe6c9491eb562c52f38ae32440cc Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 18 Jan 2018 10:13:45 -0800 Subject: [PATCH 016/192] Add statsd-service to the example kafka-monitor.properties --- config/kafka-monitor.properties | 34 ++++++++++++++----- .../StatsdMetricsReporterServiceConfig.java | 21 ------------ 2 files changed, 26 insertions(+), 29 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 60f36ac3..2ffd94a4 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -67,6 +67,16 @@ }, + "jetty-service": { + "class.name": "com.linkedin.kmf.services.JettyService", + "jetty.port": 8000 + }, + + "jolokia-service": { + "class.name": "com.linkedin.kmf.services.JolokiaService" + }, + + "reporter-service": { "class.name": "com.linkedin.kmf.services.DefaultMetricsReporterService", "report.interval.sec": 1, @@ -83,16 +93,24 @@ "kmf.services:type=produce-service,name=*:produce-error-rate", "kmf.services:type=consume-service,name=*:consume-error-rate" ] - }, + } + + +# Example statsd-service to report metrics +# "statsd-service": { +# "class.name": "com.linkedin.kmf.services.StatsdMetricsReporterService", +# "report.statsd.host": "localhost", +# "report.statsd.port": "8125", +# "report.statsd.prefix": "kafka-monitor", +# "report.interval.sec": 1, +# "report.metrics.list": [ +# "kmf:type=kafka-monitor:offline-runnable-count", +# "kmf.services:type=produce-service,name=*:produce-availability-avg", +# "kmf.services:type=consume-service,name=*:consume-availability-avg" +# ] +# } - "jetty-service": { - "class.name": "com.linkedin.kmf.services.JettyService", - "jetty.port": 8000 - }, - "jolokia-service": { - "class.name": "com.linkedin.kmf.services.JolokiaService" - } } diff --git a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java index a0b98efc..9ed2132e 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java @@ -10,27 +10,6 @@ * * In order to enable the StatsD metrics export, add the following section to kafka-monitor.properties file * - ========================================================================================== - "statsd-service": { - "class.name": "com.linkedin.kmf.services.StatsdMetricsReporterService", - "report.statsd.host": "localhost", - "report.statsd.port": "8125", - "report.statsd.prefix": "kafka-monitor", - "report.interval.sec": 1, - "report.metrics.list": [ - "kmf.services:type=produce-service,name=*:produce-availability-avg", - "kmf.services:type=consume-service,name=*:consume-availability-avg", - "kmf.services:type=produce-service,name=*:records-produced-total", - "kmf.services:type=consume-service,name=*:records-consumed-total", - "kmf.services:type=consume-service,name=*:records-lost-total", - "kmf.services:type=consume-service,name=*:records-duplicated-total", - "kmf.services:type=consume-service,name=*:records-delay-ms-avg", - "kmf.services:type=produce-service,name=*:records-produced-rate", - "kmf.services:type=produce-service,name=*:produce-error-rate", - "kmf.services:type=consume-service,name=*:consume-error-rate" - ] - } - ========================================================================================== */ package com.linkedin.kmf.services.configs; From e484b7a9d869ee6e027b833b4a20e8b38f0c7f53 Mon Sep 17 00:00:00 2001 From: hackerwin7 Date: Thu, 25 Jan 2018 10:46:22 +0800 Subject: [PATCH 017/192] Add KafkaMetricsReporterService to support send metrics to kafka (#109) --- config/kafka-monitor.properties | 22 +++ .../java/com/linkedin/kmf/common/Utils.java | 9 +- .../services/KafkaMetricsReporterService.java | 139 ++++++++++++++++++ .../KafkaMetricsReporterServiceConfig.java | 77 ++++++++++ .../kmf/topicfactory/DefaultTopicFactory.java | 2 +- 5 files changed, 244 insertions(+), 5 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java create mode 100644 src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 2ffd94a4..199e7cca 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -111,6 +111,28 @@ # } +# Example kafka-service to report metrics +# "reporter-kafka-service": { +# "class.name": "com.linkedin.kmf.services.KafkaMetricsReporterService", +# "report.interval.sec": 3, +# "zookeeper.connect": "localhost:2181", +# "bootstrap.servers": "localhost:9092", +# "topic": "kafka-monitor-topic-metrics", +# "report.kafka.topic.replication.factor": 1, +# "report.metrics.list": [ +# "kmf.services:type=produce-service,name=*:produce-availability-avg", +# "kmf.services:type=consume-service,name=*:consume-availability-avg", +# "kmf.services:type=produce-service,name=*:records-produced-total", +# "kmf.services:type=consume-service,name=*:records-consumed-total", +# "kmf.services:type=consume-service,name=*:records-lost-total", +# "kmf.services:type=consume-service,name=*:records-duplicated-total", +# "kmf.services:type=consume-service,name=*:records-delay-ms-avg", +# "kmf.services:type=produce-service,name=*:records-produced-rate", +# "kmf.services:type=produce-service,name=*:produce-error-rate", +# "kmf.services:type=consume-service,name=*:consume-error-rate" +# ] +# } + } diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 59ac0396..5399f99a 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -70,7 +70,7 @@ public static int getPartitionNumForTopic(String zkUrl, String topic) { } /** - * Create the topic that the monitor uses to monitor the cluster. This method attempts to create a topic so that all + * Create the topic. This method attempts to create a topic so that all * the brokers in the cluster will have partitionToBrokerRatio partitions. If the topic exists, but has different parameters * then this does nothing to update the parameters. * @@ -79,18 +79,19 @@ public static int getPartitionNumForTopic(String zkUrl, String topic) { * @param topic topic name * @param replicationFactor the replication factor for the topic * @param partitionToBrokerRatio This is multiplied by the number brokers to compute the number of partitions in the topic. + * @param minPartitionNum partition number to be created at least * @param topicConfig additional parameters for the topic for example min.insync.replicas * @return the number of partitions created */ - public static int createMonitoringTopicIfNotExists(String zkUrl, String topic, int replicationFactor, - double partitionToBrokerRatio, Properties topicConfig) { + public static int createTopicIfNotExists(String zkUrl, String topic, int replicationFactor, + double partitionToBrokerRatio, int minPartitionNum, Properties topicConfig) { ZkUtils zkUtils = ZkUtils.apply(zkUrl, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); try { if (AdminUtils.topicExists(zkUtils, topic)) { return getPartitionNumForTopic(zkUrl, topic); } int brokerCount = zkUtils.getAllBrokersInCluster().size(); - int partitionCount = (int) Math.ceil(brokerCount * partitionToBrokerRatio); + int partitionCount = Math.max((int) Math.ceil(brokerCount * partitionToBrokerRatio), minPartitionNum); try { AdminUtils.createTopic(zkUtils, topic, partitionCount, replicationFactor, topicConfig, RackAwareMode.Enforced$.MODULE$); diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java new file mode 100644 index 00000000..d0fd0ecf --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -0,0 +1,139 @@ +/** + * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ +package com.linkedin.kmf.services; + +import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.kmf.common.MbeanAttributeValue; +import com.linkedin.kmf.common.Utils; +import com.linkedin.kmf.services.configs.KafkaMetricsReporterServiceConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class KafkaMetricsReporterService implements Service { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricsReporterService.class); + private static final String METRICS_PRODUCER_ID = "kafka-metrics-reporter-id"; + + private final String _name; + private final List _metricsNames; + private final int _reportIntervalSec; + private final ScheduledExecutorService _executor; + + private KafkaProducer _producer; + private final String _brokerList; + private final String _topic; + + private final ObjectMapper _parser = new ObjectMapper(); + + public KafkaMetricsReporterService(Map props, String name) throws Exception { + _name = name; + KafkaMetricsReporterServiceConfig config = new KafkaMetricsReporterServiceConfig(props); + _metricsNames = config.getList(KafkaMetricsReporterServiceConfig.REPORT_METRICS_CONFIG); + _reportIntervalSec = config.getInt(KafkaMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG); + _executor = Executors.newSingleThreadScheduledExecutor(); + + _brokerList = config.getString(KafkaMetricsReporterServiceConfig.BOOTSTRAP_SERVERS_CONFIG); + initializeProducer(); + + _topic = config.getString(KafkaMetricsReporterServiceConfig.TOPIC_CONFIG); + Utils.createTopicIfNotExists(config.getString(KafkaMetricsReporterServiceConfig.ZOOKEEPER_CONNECT_CONFIG), + _topic, + config.getInt(KafkaMetricsReporterServiceConfig.TOPIC_REPLICATION_FACTOR), + 0, + 1, // fixed partition count 1 + new Properties()); + } + + @Override + public synchronized void start() { + _executor.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/KafkaMetricsReporterService failed to report metrics", e); + } + } + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); + LOG.info("{}/KafkaMetricsReporterService started", _name); + } + + @Override + public synchronized void stop() { + _executor.shutdown(); + _producer.close(); + LOG.info("{}/KafkaMetricsReporterService stopped", _name); + } + + @Override + public boolean isRunning() { + return !_executor.isShutdown(); + } + + @Override + public void awaitShutdown() { + try { + _executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.info("Thread interrupted when waiting for {}/KafkaMetricsReporterService to shutdown", _name); + } + LOG.info("{}/KafkaMetricsReporterService shutdown completed", _name); + } + + private void initializeProducer() throws Exception { + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000"); + producerProps.put(ProducerConfig.RETRIES_CONFIG, "3"); + producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, String.valueOf(Long.MAX_VALUE)); + producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, METRICS_PRODUCER_ID); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, _brokerList); + _producer = new KafkaProducer<>(producerProps); + } + + private void reportMetrics() { + Map metrics = new HashMap<>(); + for (String metricName : _metricsNames) { + String mbeanExpr = metricName.substring(0, metricName.lastIndexOf(":")); + String attributeExpr = metricName.substring(metricName.lastIndexOf(":") + 1); + List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + for (MbeanAttributeValue attributeValue : attributeValues) { + String metric = attributeValue.toString(); + String key = metric.substring(0, metric.lastIndexOf("=")); + String val = metric.substring(metric.lastIndexOf("=") + 1); + metrics.put(key, val); + } + } + try { + LOG.debug("Kafka Metrics Reporter sending metrics = " + _parser.writerWithDefaultPrettyPrinter().writeValueAsString(metrics)); + _producer.send(new ProducerRecord(_topic, _parser.writeValueAsString(metrics))); + } catch (JsonProcessingException e) { + LOG.warn("unsupported json format: " + metrics, e); + } + } +} diff --git a/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java new file mode 100644 index 00000000..d1f0b374 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java @@ -0,0 +1,77 @@ +/** + * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ +package com.linkedin.kmf.services.configs; + +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; + +import java.util.Arrays; +import java.util.Map; + +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; + +public class KafkaMetricsReporterServiceConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + public static final String REPORT_METRICS_CONFIG = CommonServiceConfig.REPORT_METRICS_CONFIG; + public static final String REPORT_METRICS_DOC = CommonServiceConfig.REPORT_METRICS_DOC; + + public static final String REPORT_INTERVAL_SEC_CONFIG = CommonServiceConfig.REPORT_INTERVAL_SEC_CONFIG; + public static final String REPORT_INTERVAL_SEC_DOC = CommonServiceConfig.REPORT_INTERVAL_SEC_DOC; + + public static final String ZOOKEEPER_CONNECT_CONFIG = CommonServiceConfig.ZOOKEEPER_CONNECT_CONFIG; + public static final String ZOOKEEPER_CONNECT_DOC = CommonServiceConfig.ZOOKEEPER_CONNECT_DOC; + + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonServiceConfig.BOOTSTRAP_SERVERS_CONFIG; + public static final String BOOTSTRAP_SERVERS_DOC = CommonServiceConfig.BOOTSTRAP_SERVERS_DOC; + + public static final String TOPIC_CONFIG = CommonServiceConfig.TOPIC_CONFIG; + public static final String TOPIC_DOC = CommonServiceConfig.TOPIC_DOC; + + public static final String TOPIC_REPLICATION_FACTOR = "report.kafka.topic.replication.factor"; + public static final String TOPIC_REPLICATION_FACTOR_DOC = "This replication factor is used to create the metrics reporter topic."; + + + static { + CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, + ConfigDef.Type.LIST, + Arrays.asList("kmf.services:*:*"), + ConfigDef.Importance.MEDIUM, + REPORT_METRICS_DOC) + .define(REPORT_INTERVAL_SEC_CONFIG, + ConfigDef.Type.INT, + 1, + ConfigDef.Importance.LOW, + REPORT_INTERVAL_SEC_DOC) + .define(ZOOKEEPER_CONNECT_CONFIG, + ConfigDef.Type.STRING, + ConfigDef.Importance.HIGH, + ZOOKEEPER_CONNECT_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, + ConfigDef.Type.STRING, + ConfigDef.Importance.HIGH, + BOOTSTRAP_SERVERS_DOC) + .define(TOPIC_CONFIG, + ConfigDef.Type.STRING, + ConfigDef.Importance.HIGH, + TOPIC_DOC) + .define(TOPIC_REPLICATION_FACTOR, + ConfigDef.Type.INT, + 1, + atLeast(1), + ConfigDef.Importance.LOW, + TOPIC_REPLICATION_FACTOR_DOC); + } + + public KafkaMetricsReporterServiceConfig(Map props) { + super(CONFIG, props); + } +} diff --git a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java index 5158d24d..82325e41 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java @@ -23,6 +23,6 @@ public DefaultTopicFactory(Map config) { @Override public int createTopicIfNotExist(String zkUrl, String topic, int replicationFactor, double partitionToBrokerRatio, Properties topicConfig) { - return Utils.createMonitoringTopicIfNotExists(zkUrl, topic, replicationFactor, partitionToBrokerRatio, topicConfig); + return Utils.createTopicIfNotExists(zkUrl, topic, replicationFactor, partitionToBrokerRatio, 1, topicConfig); } } From 132a5321caa764652809f3aec9b6a31bb9d4ca6d Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 12 Feb 2018 17:17:15 -0800 Subject: [PATCH 018/192] Upgrade kafka dependency to 0.11.0.1 (#93) --- build.gradle | 4 ++-- .../com/linkedin/kmf/services/ConsumeService.java | 11 ++++------- .../com/linkedin/kmf/services/ProduceService.java | 8 ++++---- .../kmf/services/configs/CommonServiceConfig.java | 2 +- 4 files changed, 11 insertions(+), 14 deletions(-) diff --git a/build.gradle b/build.gradle index a00873d6..0eaa5d23 100644 --- a/build.gradle +++ b/build.gradle @@ -16,8 +16,8 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.11:0.10.2.1' - compile 'org.apache.kafka:kafka-clients:0.10.2.1' + compile 'org.apache.kafka:kafka_2.11:0.11.0.1' + compile 'org.apache.kafka:kafka-clients:0.11.0.1' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index e2d07f77..b493f401 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -215,7 +215,6 @@ public boolean isRunning() { } private class ConsumeMetrics { - public final Metrics metrics; private final Sensor _bytesConsumed; private final Sensor _consumeError; private final Sensor _recordsConsumed; @@ -224,9 +223,7 @@ private class ConsumeMetrics { private final Sensor _recordsDelay; private final Sensor _recordsDelayed; - public ConsumeMetrics(Metrics metrics, final Map tags) { - this.metrics = metrics; - + public ConsumeMetrics(final Metrics metrics, final Map tags) { _bytesConsumed = metrics.sensor("bytes-consumed"); _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); @@ -265,9 +262,9 @@ public ConsumeMetrics(Metrics metrics, final Map tags) { new Measurable() { @Override public double measure(MetricConfig config, long now) { - double recordsConsumedRate = _sensors.metrics.metrics().get(new MetricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsLostRate = _sensors.metrics.metrics().get(new MetricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsDelayedRate = _sensors.metrics.metrics().get(new MetricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); if (new Double(recordsLostRate).isNaN()) recordsLostRate = 0; diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 0f0f481d..19d58ecb 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -131,7 +131,7 @@ private void initializeProducer() throws Exception { producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000"); producerProps.put(ProducerConfig.RETRIES_CONFIG, "3"); - producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true"); + producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MAX_VALUE); producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); @@ -219,7 +219,7 @@ private class ProduceMetrics { private final ConcurrentMap _produceErrorPerPartition; private final Map _tags; - public ProduceMetrics(Metrics metrics, final Map tags) { + public ProduceMetrics(final Metrics metrics, final Map tags) { this.metrics = metrics; this._tags = tags; @@ -241,8 +241,8 @@ public double measure(MetricConfig config, long now) { double availabilitySum = 0.0; int partitionNum = _partitionNum.get(); for (int partition = 0; partition < partitionNum; partition++) { - double recordsProduced = _sensors.metrics.metrics().get(new MetricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); - double produceError = _sensors.metrics.metrics().get(new MetricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); + double recordsProduced = metrics.metrics().get(metrics.metricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); + double produceError = metrics.metrics().get(metrics.metricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); // If there is no error, error rate sensor may expire and the value may be NaN. Treat NaN as 0 for error rate. if (Double.isNaN(produceError) || Double.isInfinite(produceError)) { produceError = 0; diff --git a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java index 71c05c23..654685a6 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java @@ -17,7 +17,7 @@ public class CommonServiceConfig { public static final String ZOOKEEPER_CONNECT_DOC = "Zookeeper connect string."; public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; - public static final String BOOTSTRAP_SERVERS_DOC = CommonClientConfigs.BOOSTRAP_SERVERS_DOC; + public static final String BOOTSTRAP_SERVERS_DOC = CommonClientConfigs.BOOTSTRAP_SERVERS_DOC; public static final String TOPIC_CONFIG = "topic"; public static final String TOPIC_DOC = "Topic to be used by the service."; From b8e0a72777e6e07171d6536a0dda249a2966bae9 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 18 Apr 2018 14:05:50 -0700 Subject: [PATCH 019/192] Add metrics to measure produce delay (#117) --- .../com/linkedin/kmf/services/ProduceService.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 19d58ecb..39143668 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -39,6 +39,8 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Total; import org.apache.kafka.common.utils.SystemTime; @@ -215,6 +217,7 @@ private class ProduceMetrics { public final Metrics metrics; private final Sensor _recordsProduced; private final Sensor _produceError; + private final Sensor _produceDelay; private final ConcurrentMap _recordsProducedPerPartition; private final ConcurrentMap _produceErrorPerPartition; private final Map _tags; @@ -234,6 +237,10 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { _produceError.add(new MetricName("produce-error-rate", METRIC_GROUP_NAME, "The average number of errors per second", tags), new Rate()); _produceError.add(new MetricName("produce-error-total", METRIC_GROUP_NAME, "The total number of errors", tags), new Total()); + _produceDelay = metrics.sensor("produce-delay"); + _produceDelay.add(new MetricName("produce-delay-ms-avg", METRIC_GROUP_NAME, "The average delay in ms for produce request", tags), new Avg()); + _produceDelay.add(new MetricName("produce-delay-ms-max", METRIC_GROUP_NAME, "The maximum delay in ms for produce request", tags), new Max()); + metrics.addMetric(new MetricName("produce-availability-avg", METRIC_GROUP_NAME, "The average produce availability", tags), new Measurable() { @Override @@ -295,9 +302,11 @@ private class ProduceRunnable implements Runnable { public void run() { try { long nextIndex = _nextIndexPerPartition.get(_partition).get(); - String message = Utils.jsonFromFields(_topic, nextIndex, System.currentTimeMillis(), _producerId, _recordSize); + long currMs = System.currentTimeMillis(); + String message = Utils.jsonFromFields(_topic, nextIndex, currMs, _producerId, _recordSize); BaseProducerRecord record = new BaseProducerRecord(_topic, _partition, _key, message); RecordMetadata metadata = _producer.send(record, _sync); + _sensors._produceDelay.record(System.currentTimeMillis() - currMs); _sensors._recordsProduced.record(); _sensors._recordsProducedPerPartition.get(_partition).record(); if (nextIndex == -1 && _sync) { From c0ae4f34a5e4125b225b240a067d45931328622a Mon Sep 17 00:00:00 2001 From: Javier Li Sam Date: Tue, 1 May 2018 11:28:34 -0700 Subject: [PATCH 020/192] Typo in readme - Missing the letter `a` in `use-case` --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 5deceb80..623c5697 100644 --- a/README.md +++ b/README.md @@ -48,7 +48,7 @@ ConsumeServiceConfig.java. - You can specify multiple SingleClusterMonitor in the kafka-monitor.properties to monitor multiple Kafka clusters in one Kafka Monitor process. As another -advanced use-cse, you can point ProduceService and ConsumeService to two +advanced use-case, you can point ProduceService and ConsumeService to two different Kafka clusters that are connected by MirrorMaker to monitor their end-to-end latency. From d7e1f46e4c6cea95e564a6b60044bd2bf0495bcf Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 22 May 2018 13:21:04 -0700 Subject: [PATCH 021/192] Update branch list in the README --- README.md | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 623c5697..b882d105 100644 --- a/README.md +++ b/README.md @@ -25,14 +25,12 @@ the monitor topic. Kafka Monitor requires Gradle 2.0 or higher. Java 7 should be used for building in order to support both Java 7 and Java 8 at runtime. -Kafka Monitor supports Apache Kafka 0.8 to 0.10: +Kafka Monitor supports Apache Kafka 0.8 to 0.11: - Use branch 0.8.2.2 to work with Apache Kafka 0.8 - Use branch 0.9.0.1 to work with Apache Kafka 0.9 -- Use master branch to work with Apache Kafka 0.10.2.1 +- Use branch 0.10.2.1 to work with Apache Kafka 0.10 +- Use master branch to work with Apache Kafka 0.11 -Kafka Monitor supports Apache Kafka 0.8 and 0.9. Use branch 0.8.2.2 to monitor Apache -Kafka cluster 0.8. Use branch 0.9.0.1 to compile with Kafka 0.9. Use master -branch to compile with Kafka 0.10. ### Configuration Tips From b767c663bfcb795a11d7eafd345a9cb6379c7492 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 23 May 2018 12:22:16 -0700 Subject: [PATCH 022/192] Add 99th and 999th percentile produce delay metrics (#122) --- .../linkedin/kmf/services/ProduceService.java | 14 +++++++++++++- .../services/configs/ProduceServiceConfig.java | 18 ++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 39143668..58eeca20 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -41,6 +41,8 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Percentile; +import org.apache.kafka.common.metrics.stats.Percentiles; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Total; import org.apache.kafka.common.utils.SystemTime; @@ -77,6 +79,8 @@ public class ProduceService implements Service { private final int _threadsNum; private final String _zkConnect; private final boolean _treatZeroThroughputAsUnavailable; + private final int _latencyPercentileMaxMs; + private final int _latencyPercentileGranularityMs; public ProduceService(Map props, String name) throws Exception { _name = name; @@ -84,7 +88,8 @@ public ProduceService(Map props, String name) throws Exception { _zkConnect = config.getString(ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG); _brokerList = config.getString(ProduceServiceConfig.BOOTSTRAP_SERVERS_CONFIG); String producerClass = config.getString(ProduceServiceConfig.PRODUCER_CLASS_CONFIG); - + _latencyPercentileMaxMs = config.getInt(ProduceServiceConfig.LATENCY_PERCENTILE_MAX_MS_CONFIG); + _latencyPercentileGranularityMs = config.getInt(ProduceServiceConfig.LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG); _partitioner = config.getConfiguredInstance(ProduceServiceConfig.PARTITIONER_CLASS_CONFIG, KMPartitioner.class); _threadsNum = config.getInt(ProduceServiceConfig.PRODUCE_THREAD_NUM_CONFIG); _topic = config.getString(ProduceServiceConfig.TOPIC_CONFIG); @@ -241,6 +246,13 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { _produceDelay.add(new MetricName("produce-delay-ms-avg", METRIC_GROUP_NAME, "The average delay in ms for produce request", tags), new Avg()); _produceDelay.add(new MetricName("produce-delay-ms-max", METRIC_GROUP_NAME, "The maximum delay in ms for produce request", tags), new Max()); + // There are 2 extra buckets use for values smaller than 0.0 or larger than max, respectively. + int bucketNum = _latencyPercentileMaxMs / _latencyPercentileGranularityMs + 2; + int sizeInBytes = 4 * bucketNum; + _produceDelay.add(new Percentiles(sizeInBytes, _latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, + new Percentile(new MetricName("produce-delay-ms-99th", METRIC_GROUP_NAME, "The 99th percentile delay in ms for produce request", tags), 99.0), + new Percentile(new MetricName("produce-delay-ms-999th", METRIC_GROUP_NAME, "The 999th percentile delay in ms for produce request", tags), 99.9))); + metrics.addMetric(new MetricName("produce-availability-avg", METRIC_GROUP_NAME, "The average produce availability", tags), new Measurable() { @Override diff --git a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java index b10894a8..fcf8bb13 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java @@ -55,6 +55,14 @@ public class ProduceServiceConfig extends AbstractConfig { public static final String PRODUCER_PROPS_CONFIG = "produce.producer.props"; public static final String PRODUCER_PROPS_DOC = "The properties used to config producer in produce service."; + public static final String LATENCY_PERCENTILE_MAX_MS_CONFIG = "produce.latency.percentile.max.ms"; + public static final String LATENCY_PERCENTILE_MAX_MS_DOC = "This is used to derive the bucket number used to configure latency percentile metric. " + + "Any latency larger than this max value will be rounded down to the max value."; + + public static final String LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG = "produce.latency.percentile.granularity.ms"; + public static final String LATENCY_PERCENTILE_GRANULARITY_MS_DOC = "This is used to derive the bucket number used to configure latency percentile metric. " + + "The latency at the specified percentile should be multiple of this value."; + public static final String PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_CONFIG = "produce.treat.zero.throughput.as.unavailable"; public static final String PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_DOC = "If it is set to true, produce availability is set to 0 " + "if no message can be produced, regardless of whether there is exception. If this is set to false, availability will only drop below 1 if there is exception " + @@ -109,6 +117,16 @@ public class ProduceServiceConfig extends AbstractConfig { true, ConfigDef.Importance.MEDIUM, PRODUCER_TREAT_ZERO_THROUGHPUT_AS_UNAVAILABLE_DOC) + .define(LATENCY_PERCENTILE_MAX_MS_CONFIG, + ConfigDef.Type.INT, + 5000, + ConfigDef.Importance.LOW, + LATENCY_PERCENTILE_MAX_MS_DOC) + .define(LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG, + ConfigDef.Type.INT, + 1, + ConfigDef.Importance.LOW, + LATENCY_PERCENTILE_GRANULARITY_MS_DOC) .define(PRODUCE_THREAD_NUM_CONFIG, ConfigDef.Type.INT, 5, From 359cd34e587d076f720ee59d99d93d665ca2e82d Mon Sep 17 00:00:00 2001 From: Le Yu Date: Wed, 30 May 2018 03:02:47 +0800 Subject: [PATCH 023/192] Export to prometheus supported format using jmx exporter with this config file (#120) --- config/prometheus-exporter.yaml | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 config/prometheus-exporter.yaml diff --git a/config/prometheus-exporter.yaml b/config/prometheus-exporter.yaml new file mode 100644 index 00000000..7401e3a4 --- /dev/null +++ b/config/prometheus-exporter.yaml @@ -0,0 +1,7 @@ +--- +lowercaseOutputName: true +rules: +- pattern : kmf<>([\w\d-]+) + name: kmf_$1_$2 +- pattern : kmf.services<>([\w\d-]+) + name: kmf_services_$1_$2_$3 From d84da596673eea35c08d340577b9fa26344103c4 Mon Sep 17 00:00:00 2001 From: Javier Li Sam Date: Tue, 5 Jun 2018 12:33:13 -0700 Subject: [PATCH 024/192] Fixes double prefixing of metric names (#123) The current statsd client already appends the prefix at the beginning of the metric name. Unfortunately the current code was also appending the prefix. For example: If `prefix` was `kafka-monitor`, the final metric name produced would be: `kafka-monitor.kafka-monitor.produce_service.single-cluster-monitor.produce-availability-avg` --- .../kmf/services/StatsdMetricsReporterService.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java index 9838b350..a0b2bad7 100644 --- a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java @@ -34,7 +34,6 @@ public class StatsdMetricsReporterService implements Service { private final int _reportIntervalSec; private final ScheduledExecutorService _executor; private final StatsDClient _statsdClient; - private final String _metricNamePrefix; public StatsdMetricsReporterService(Map props, String name) { StatsdMetricsReporterServiceConfig config = new StatsdMetricsReporterServiceConfig(props); @@ -43,8 +42,7 @@ public StatsdMetricsReporterService(Map props, String name) { _metricNames = config.getList(StatsdMetricsReporterServiceConfig.REPORT_METRICS_CONFIG); _reportIntervalSec = config.getInt(StatsdMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG); _executor = Executors.newSingleThreadScheduledExecutor(); - _metricNamePrefix = config.getString(StatsdMetricsReporterServiceConfig.REPORT_STATSD_PREFIX); - _statsdClient = new NonBlockingStatsDClient(_metricNamePrefix, + _statsdClient = new NonBlockingStatsDClient(config.getString(StatsdMetricsReporterServiceConfig.REPORT_STATSD_PREFIX), config.getString(StatsdMetricsReporterServiceConfig.REPORT_STATSD_HOST), config.getInt(StatsdMetricsReporterServiceConfig.REPORT_STATSD_PORT)); } @@ -91,10 +89,8 @@ private String generateStatsdMetricName(String bean, String attribute) { String service = bean.split(":")[1]; String serviceName = service.split(",")[0].split("=")[1]; String serviceType = service.split(",")[1].split("=")[1]; - String[] segs = {_metricNamePrefix, serviceType, serviceName, attribute}; - String metricName = StringUtils.join(segs, "."); - - return _metricNamePrefix.isEmpty() ? metricName.substring(1) : metricName; + String[] segs = {serviceType, serviceName, attribute}; + return StringUtils.join(segs, "."); } private void reportMetrics() { From f8fa763f2bebebba6af317cc598e3aced5149cc1 Mon Sep 17 00:00:00 2001 From: Jonathan Santilli Date: Tue, 19 Jun 2018 04:07:50 +0100 Subject: [PATCH 025/192] Fix typo (#125) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b882d105..1b4aceff 100644 --- a/README.md +++ b/README.md @@ -76,7 +76,7 @@ $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties ``` ### Run Kafka Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) -Edit `config/kafka-monitor.properties` to specify custom configurations for prodcuer in the key/value map `produce.producer.props` in +Edit `config/kafka-monitor.properties` to specify custom configurations for producer in the key/value map `produce.producer.props` in `config/kafka-monitor.properties`. Similarly specify configurations for consumer as well. The documentation for producer and consumer in the key/value maps can be found in the Apache Kafka wiki. From f37f0e452ba040ce5f83e43940b9db5c6f311f0f Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 9 Jul 2018 17:34:30 -0700 Subject: [PATCH 026/192] Upgrade to kafka 1.0.1 (#127) --- README.md | 3 ++- build.gradle | 6 ++--- .../MultiClusterTopicManagementService.java | 26 ++++++++++++++++--- .../services/TopicManagementServiceTest.java | 2 +- 4 files changed, 29 insertions(+), 8 deletions(-) diff --git a/README.md b/README.md index 1b4aceff..4040b24f 100644 --- a/README.md +++ b/README.md @@ -29,7 +29,8 @@ Kafka Monitor supports Apache Kafka 0.8 to 0.11: - Use branch 0.8.2.2 to work with Apache Kafka 0.8 - Use branch 0.9.0.1 to work with Apache Kafka 0.9 - Use branch 0.10.2.1 to work with Apache Kafka 0.10 -- Use master branch to work with Apache Kafka 0.11 +- Use branch 0.11.x to work with Apache Kafka 0.11 +- Use master branch to work with Apache Kafka 1.0 ### Configuration Tips diff --git a/build.gradle b/build.gradle index 0eaa5d23..5db03e8d 100644 --- a/build.gradle +++ b/build.gradle @@ -16,8 +16,8 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.11:0.11.0.1' - compile 'org.apache.kafka:kafka-clients:0.11.0.1' + compile 'org.apache.kafka:kafka_2.11:1.0.1' + compile 'org.apache.kafka:kafka-clients:1.0.1' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' @@ -69,5 +69,5 @@ allprojects { } task wrapper(type: Wrapper) { - gradleVersion = '2.11' + gradleVersion = '4.8.1' } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index ae77fdd2..fb324894 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -33,7 +33,6 @@ import kafka.admin.AdminUtils; import kafka.admin.BrokerMetadata; import kafka.admin.PreferredReplicaLeaderElectionCommand; -import kafka.admin.RackAwareMode; import kafka.cluster.Broker; import kafka.common.TopicAndPartition; import kafka.server.ConfigType; @@ -220,11 +219,15 @@ int minPartitionNum() { void maybeAddPartitions(int minPartitionNum) { ZkUtils zkUtils = ZkUtils.apply(_zkConnect, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); try { - int partitionNum = getPartitionInfo(zkUtils, _topic).size(); + scala.collection.Map> existingAssignment = getPartitionAssignment(zkUtils, _topic); + int partitionNum = existingAssignment.size(); + if (partitionNum < minPartitionNum) { LOG.info("MultiClusterTopicManagementService will increase partition of the topic {} " + "in cluster {} from {} to {}.", _topic, _zkConnect, partitionNum, minPartitionNum); - AdminUtils.addPartitions(zkUtils, _topic, minPartitionNum, null, false, RackAwareMode.Enforced$.MODULE$); + + scala.Option>> replicaAssignment = scala.Option.apply(null); + AdminUtils.addPartitions(zkUtils, _topic, existingAssignment, getAllBrokers(zkUtils), minPartitionNum, replicaAssignment, false); } } finally { zkUtils.close(); @@ -313,6 +316,23 @@ private static void reassignPartitions(ZkUtils zkUtils, Collection broke zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath(), newAssignmentJson, zkUtils.DefaultAcls()); } + private static scala.collection.mutable.ArrayBuffer getAllBrokers(ZkUtils zkUtils) { + Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkUtils.getAllBrokersInCluster()); + scala.collection.mutable.ArrayBuffer brokersMetadata = new scala.collection.mutable.ArrayBuffer<>(brokers.size()); + for (Broker broker : brokers) { + brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), broker.rack())); + } + return brokersMetadata; + } + + private static scala.collection.Map> getPartitionAssignment(ZkUtils zkUtils, String topic) { + scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); + topicList.$plus$eq(topic); + scala.collection.Map> partitionAssignment = + zkUtils.getPartitionAssignmentForTopics(topicList).apply(topic); + return partitionAssignment; + } + private static List getPartitionInfo(ZkUtils zkUtils, String topic) { scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); topicList.$plus$eq(topic); diff --git a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java index 97f6d5c7..5d08e3c4 100644 --- a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java @@ -14,7 +14,7 @@ import kafka.cluster.Broker; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.security.auth.SecurityProtocol; import org.testng.Assert; import org.testng.annotations.Test; import com.linkedin.kmf.services.MultiClusterTopicManagementService.TopicManagementHelper; From 973bbb4fd7d253b43453ac4dd1fd062d0d05ecfb Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 17 Aug 2018 17:29:52 -0700 Subject: [PATCH 027/192] Upgrade to kafka 1.1.1 (#131) * Upgrade to kafka 1.1.1 --- README.md | 3 +- build.gradle | 7 +- .../MultiClusterTopicManagementService.java | 105 +++++++++--------- 3 files changed, 61 insertions(+), 54 deletions(-) diff --git a/README.md b/README.md index 4040b24f..cd298ea2 100644 --- a/README.md +++ b/README.md @@ -30,7 +30,8 @@ Kafka Monitor supports Apache Kafka 0.8 to 0.11: - Use branch 0.9.0.1 to work with Apache Kafka 0.9 - Use branch 0.10.2.1 to work with Apache Kafka 0.10 - Use branch 0.11.x to work with Apache Kafka 0.11 -- Use master branch to work with Apache Kafka 1.0 +- Use branch 1.0.x to work with Apache Kafka 1.0 +- Use master branch to work with Apache Kafka 1.1 ### Configuration Tips diff --git a/build.gradle b/build.gradle index 5db03e8d..a12b03de 100644 --- a/build.gradle +++ b/build.gradle @@ -6,7 +6,8 @@ allprojects { version = "1.0.0" - sourceCompatibility = 1.7 + sourceCompatibility = 8 + targetCompatibility = 8 repositories { mavenCentral() @@ -16,8 +17,8 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.11:1.0.1' - compile 'org.apache.kafka:kafka-clients:1.0.1' + compile 'org.apache.kafka:kafka_2.12:1.1.1' + compile 'org.apache.kafka:kafka-clients:1.1.1' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index fb324894..0882e368 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -33,15 +33,18 @@ import kafka.admin.AdminUtils; import kafka.admin.BrokerMetadata; import kafka.admin.PreferredReplicaLeaderElectionCommand; +import kafka.admin.RackAwareMode; import kafka.cluster.Broker; -import kafka.common.TopicAndPartition; import kafka.server.ConfigType; -import kafka.utils.ZkUtils; +import kafka.zk.AdminZkClient; +import kafka.zk.KafkaZkClient; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.collection.Seq; @@ -65,6 +68,7 @@ */ public class MultiClusterTopicManagementService implements Service { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); + private static final String METRIC_GROUP_NAME = "topic-management-service"; private final AtomicBoolean _isRunning = new AtomicBoolean(false); private final String _serviceName; @@ -166,8 +170,10 @@ public void run() { LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); } } - } catch (Exception e) { - LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", e); + } catch (Throwable t) { + // Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime + // and such error is not caught by compilation + LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); stop(); } } @@ -217,9 +223,12 @@ int minPartitionNum() { } void maybeAddPartitions(int minPartitionNum) { - ZkUtils zkUtils = ZkUtils.apply(_zkConnect, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); + KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, + ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); + AdminZkClient adminZkClient = new AdminZkClient(zkClient); + try { - scala.collection.Map> existingAssignment = getPartitionAssignment(zkUtils, _topic); + scala.collection.Map> existingAssignment = getPartitionAssignment(zkClient, _topic); int partitionNum = existingAssignment.size(); if (partitionNum < minPartitionNum) { @@ -227,19 +236,21 @@ void maybeAddPartitions(int minPartitionNum) { + "in cluster {} from {} to {}.", _topic, _zkConnect, partitionNum, minPartitionNum); scala.Option>> replicaAssignment = scala.Option.apply(null); - AdminUtils.addPartitions(zkUtils, _topic, existingAssignment, getAllBrokers(zkUtils), minPartitionNum, replicaAssignment, false); + scala.Option> brokerList = scala.Option.apply(null); + adminZkClient.addPartitions(_topic, existingAssignment, adminZkClient.getBrokerMetadatas(RackAwareMode.Disabled$.MODULE$, brokerList), minPartitionNum, replicaAssignment, false); } } finally { - zkUtils.close(); + zkClient.close(); } } void maybeReassignPartitionAndElectLeader() throws Exception { - ZkUtils zkUtils = ZkUtils.apply(_zkConnect, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); + KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, + ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); try { - List partitionInfoList = getPartitionInfo(zkUtils, _topic); - Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkUtils.getAllBrokersInCluster()); + List partitionInfoList = getPartitionInfo(zkClient, _topic); + Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkClient.getAllBrokersInCluster()); if (partitionInfoList.size() == 0) throw new IllegalStateException("Topic " + _topic + " does not exist in cluster " + _zkConnect); @@ -251,14 +262,14 @@ void maybeReassignPartitionAndElectLeader() throws Exception { LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster {}", _replicationFactor, currentReplicationFactor, _topic, _zkConnect); - if (expectedReplicationFactor > currentReplicationFactor && zkUtils.getPartitionsBeingReassigned().isEmpty()) { + if (expectedReplicationFactor > currentReplicationFactor && !zkClient.reassignPartitionsInProgress()) { LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}" + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, expectedReplicationFactor); - reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); } // Update the properties of the monitor topic if any config is different from the user-specified config - Properties currentProperties = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic(), _topic); + Properties currentProperties = zkClient.getEntityConfigs(ConfigType.Topic(), _topic); Properties expectedProperties = new Properties(); for (Object key: currentProperties.keySet()) expectedProperties.put(key, currentProperties.get(key)); @@ -268,82 +279,76 @@ void maybeReassignPartitionAndElectLeader() throws Exception { if (!currentProperties.equals(expectedProperties)) { LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + "in cluster {} from {} to {}.", _topic, _zkConnect, currentProperties, expectedProperties); - AdminUtils.changeTopicConfig(zkUtils, _topic, expectedProperties); + zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); } if (partitionInfoList.size() >= brokers.size() && someBrokerNotPreferredLeader(partitionInfoList, brokers) && - zkUtils.getPartitionsBeingReassigned().isEmpty()) { + !zkClient.reassignPartitionsInProgress()) { LOG.info("MultiClusterTopicManagementService will reassign partitions of the topic {} in cluster {}", _topic, _zkConnect); - reassignPartitions(zkUtils, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); } if (partitionInfoList.size() >= brokers.size() && someBrokerNotElectedLeader(partitionInfoList, brokers)) { LOG.info("MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in cluster {}", _topic, _zkConnect); - triggerPreferredLeaderElection(zkUtils, partitionInfoList); + triggerPreferredLeaderElection(zkClient, partitionInfoList); } } finally { - zkUtils.close(); + zkClient.close(); } } - private static void triggerPreferredLeaderElection(ZkUtils zkUtils, List partitionInfoList) { - scala.collection.mutable.HashSet scalaPartitionInfoSet = new scala.collection.mutable.HashSet<>(); + private static void triggerPreferredLeaderElection(KafkaZkClient zkClient, List partitionInfoList) { + scala.collection.mutable.HashSet scalaPartitionInfoSet = new scala.collection.mutable.HashSet<>(); for (PartitionInfo javaPartitionInfo : partitionInfoList) { - scalaPartitionInfoSet.add(new TopicAndPartition(javaPartitionInfo.topic(), javaPartitionInfo.partition())); + scalaPartitionInfoSet.add(new TopicPartition(javaPartitionInfo.topic(), javaPartitionInfo.partition())); } - PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkUtils, scalaPartitionInfoSet); + PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkClient, scalaPartitionInfoSet); } - private static void reassignPartitions(ZkUtils zkUtils, Collection brokers, String topic, int partitionCount, int replicationFactor) { + private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { scala.collection.mutable.ArrayBuffer brokersMetadata = new scala.collection.mutable.ArrayBuffer<>(brokers.size()); for (Broker broker : brokers) { brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), broker.rack())); } - scala.collection.Map> newAssignment = + scala.collection.Map> assignedReplicas = AdminUtils.assignReplicasToBrokers(brokersMetadata, partitionCount, replicationFactor, 0, 0); - scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); - topicList.$plus$eq(topic); - scala.collection.Map> currentAssignment = zkUtils.getPartitionAssignmentForTopics(topicList).apply(topic); + scala.collection.immutable.Map> newAssignment = new scala.collection.immutable.HashMap<>(); + scala.collection.Iterator>> it = assignedReplicas.iterator(); + while (it.hasNext()) { + scala.Tuple2> scalaTuple = it.next(); + TopicPartition tp = new TopicPartition(topic, (Integer) scalaTuple._1); + newAssignment = newAssignment.$plus(new scala.Tuple2<>(tp, scalaTuple._2)); + } + + scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); + scala.collection.Map> currentAssignment = zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); String currentAssignmentJson = formatAsReassignmentJson(topic, currentAssignment); - String newAssignmentJson = formatAsReassignmentJson(topic, newAssignment); + String newAssignmentJson = formatAsReassignmentJson(topic, assignedReplicas); LOG.info("Reassign partitions for topic " + topic); LOG.info("Current partition replica assignment " + currentAssignmentJson); LOG.info("New partition replica assignment " + newAssignmentJson); - zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath(), newAssignmentJson, zkUtils.DefaultAcls()); - } - - private static scala.collection.mutable.ArrayBuffer getAllBrokers(ZkUtils zkUtils) { - Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkUtils.getAllBrokersInCluster()); - scala.collection.mutable.ArrayBuffer brokersMetadata = new scala.collection.mutable.ArrayBuffer<>(brokers.size()); - for (Broker broker : brokers) { - brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), broker.rack())); - } - return brokersMetadata; + zkClient.createPartitionReassignment(newAssignment); } - private static scala.collection.Map> getPartitionAssignment(ZkUtils zkUtils, String topic) { - scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); - topicList.$plus$eq(topic); - scala.collection.Map> partitionAssignment = - zkUtils.getPartitionAssignmentForTopics(topicList).apply(topic); - return partitionAssignment; + private static scala.collection.Map> getPartitionAssignment(KafkaZkClient zkClient, String topic) { + scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); + return zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); } - private static List getPartitionInfo(ZkUtils zkUtils, String topic) { - scala.collection.mutable.ArrayBuffer topicList = new scala.collection.mutable.ArrayBuffer<>(); - topicList.$plus$eq(topic); + private static List getPartitionInfo(KafkaZkClient zkClient, String topic) { + scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); scala.collection.Map> partitionAssignments = - zkUtils.getPartitionAssignmentForTopics(topicList).apply(topic); + zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); List partitionInfoList = new ArrayList<>(); scala.collection.Iterator>> it = partitionAssignments.iterator(); while (it.hasNext()) { scala.Tuple2> scalaTuple = it.next(); Integer partition = (Integer) scalaTuple._1(); - scala.Option leaderOption = zkUtils.getLeaderForPartition(topic, partition); + scala.Option leaderOption = zkClient.getLeaderForPartition(new TopicPartition(topic, partition)); Node leader = leaderOption.isEmpty() ? null : new Node((Integer) leaderOption.get(), "", -1); Node[] replicas = new Node[scalaTuple._2().size()]; for (int i = 0; i < replicas.length; i++) { From abec62aa37c3b2942bfdaafd9ebac8648002b107 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 20 Aug 2018 09:38:16 -0700 Subject: [PATCH 028/192] Upgrade to kafka 2.0.0 (#132) --- build.gradle | 4 +- .../linkedin/kmf/consumer/OldConsumer.java | 53 ------------------- .../linkedin/kmf/services/ConsumeService.java | 6 --- .../configs/ConsumeServiceConfig.java | 2 +- 4 files changed, 3 insertions(+), 62 deletions(-) delete mode 100644 src/main/java/com/linkedin/kmf/consumer/OldConsumer.java diff --git a/build.gradle b/build.gradle index a12b03de..c570451e 100644 --- a/build.gradle +++ b/build.gradle @@ -17,8 +17,8 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.12:1.1.1' - compile 'org.apache.kafka:kafka-clients:1.1.1' + compile 'org.apache.kafka:kafka_2.12:2.0.0' + compile 'org.apache.kafka:kafka-clients:2.0.0' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' diff --git a/src/main/java/com/linkedin/kmf/consumer/OldConsumer.java b/src/main/java/com/linkedin/kmf/consumer/OldConsumer.java deleted file mode 100644 index fdd842cc..00000000 --- a/src/main/java/com/linkedin/kmf/consumer/OldConsumer.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this - * file except in compliance with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - */ -package com.linkedin.kmf.consumer; - -import kafka.consumer.Consumer; -import kafka.consumer.ConsumerConfig; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaStream; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/* - * Wrap around the old consumer from Apache Kafka and implement the #KMBaseConsumer interface - */ -public class OldConsumer implements KMBaseConsumer { - - private final ConsumerConnector _connector; - private final ConsumerIterator _iter; - - public OldConsumer(String topic, Properties consumerProperties) { - _connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProperties)); - Map topicCountMap = new HashMap<>(); - topicCountMap.put(topic, 1); - Map>> kafkaStreams = _connector.createMessageStreams(topicCountMap, new StringDecoder(null), new StringDecoder(null)); - _iter = kafkaStreams.get(topic).get(0).iterator(); - } - - @Override - public BaseConsumerRecord receive() { - if (!_iter.hasNext()) - return null; - MessageAndMetadata record = _iter.next(); - return new BaseConsumerRecord(record.topic(), record.partition(), record.offset(), record.key(), record.message()); - } - - @Override - public void close() { - _connector.shutdown(); - } - -} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index b493f401..689fb79b 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -14,7 +14,6 @@ import com.linkedin.kmf.consumer.BaseConsumerRecord; import com.linkedin.kmf.consumer.KMBaseConsumer; import com.linkedin.kmf.consumer.NewConsumer; -import com.linkedin.kmf.consumer.OldConsumer; import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import java.util.ArrayList; import java.util.HashMap; @@ -93,11 +92,6 @@ public ConsumeService(Map props, String name) throws Exception { if (consumerClassName.equals(NewConsumer.class.getCanonicalName()) || consumerClassName.equals(NewConsumer.class.getSimpleName())) { consumerClassName = NewConsumer.class.getCanonicalName(); - } else if (consumerClassName.equals(OldConsumer.class.getCanonicalName()) || consumerClassName.equals(OldConsumer.class.getSimpleName())) { - consumerClassName = OldConsumer.class.getCanonicalName(); - // The name/value of these configs are changed in the new consumer. - consumerProps.put("auto.commit.enable", "false"); - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "largest"); } // Assign config specified for ConsumeService. diff --git a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java index dd58f6f4..6a003c3c 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java @@ -29,7 +29,7 @@ public class ConsumeServiceConfig extends AbstractConfig { public static final String CONSUMER_CLASS_CONFIG = "consume.consumer.class"; public static final String CONSUMER_CLASS_DOC = "Consumer class that will be instantiated as consumer in the consume service. " - + "It can be NewConsumer, OldConsumer, or full class name of any class that implements the KMBaseConsumer interface."; + + "It can be NewConsumer or full class name of any class that implements the KMBaseConsumer interface."; public static final String LATENCY_PERCENTILE_MAX_MS_CONFIG = "consume.latency.percentile.max.ms"; public static final String LATENCY_PERCENTILE_MAX_MS_DOC = "This is used to derive the bucket number used to configure latency percentile metric. " From 6d3e9cfbad897e2599807072030d0cbfcf1a9ec8 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 20 Aug 2018 09:44:03 -0700 Subject: [PATCH 029/192] Update README --- README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index cd298ea2..e3a04871 100644 --- a/README.md +++ b/README.md @@ -25,13 +25,14 @@ the monitor topic. Kafka Monitor requires Gradle 2.0 or higher. Java 7 should be used for building in order to support both Java 7 and Java 8 at runtime. -Kafka Monitor supports Apache Kafka 0.8 to 0.11: +Kafka Monitor supports Apache Kafka 0.8 to 2.0: - Use branch 0.8.2.2 to work with Apache Kafka 0.8 - Use branch 0.9.0.1 to work with Apache Kafka 0.9 - Use branch 0.10.2.1 to work with Apache Kafka 0.10 - Use branch 0.11.x to work with Apache Kafka 0.11 - Use branch 1.0.x to work with Apache Kafka 1.0 -- Use master branch to work with Apache Kafka 1.1 +- Use branch 1.1.x to work with Apache Kafka 1.1 +- Use master branch to work with Apache Kafka 2.0 ### Configuration Tips From 6055aca4fa236e64679e6c495bb893814f9a6349 Mon Sep 17 00:00:00 2001 From: Rohit Vyavahare Date: Mon, 20 Aug 2018 11:47:10 -0700 Subject: [PATCH 030/192] Add support for reporting metrics to SignalFx (#128) --- build.gradle | 2 + checkstyle/checkstyle.xml | 4 + checkstyle/suppressions.xml | 9 + config/kafka-monitor.properties | 10 + .../SignalFxMetricsReporterService.java | 190 ++++++++++++++++++ .../SignalFxMetricsReporterServiceConfig.java | 61 ++++++ 6 files changed, 276 insertions(+) create mode 100644 checkstyle/suppressions.xml create mode 100644 src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java create mode 100644 src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java diff --git a/build.gradle b/build.gradle index c570451e..e896702d 100644 --- a/build.gradle +++ b/build.gradle @@ -26,6 +26,7 @@ allprojects { compile 'org.jolokia:jolokia-jvm:1.3.3' compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' + compile 'com.signalfx.public:signalfx-codahale:0.0.47' testCompile 'org.testng:testng:6.8.8' } @@ -55,6 +56,7 @@ allprojects { checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") + configProperties = [ "suppressionFile" : new File(rootDir, "checkstyle/suppressions.xml") ] } test.dependsOn('checkstyleMain', 'checkstyleTest') diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index f7edb531..e2be1a6a 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -21,6 +21,10 @@ + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml new file mode 100644 index 00000000..b5062103 --- /dev/null +++ b/checkstyle/suppressions.xml @@ -0,0 +1,9 @@ + + + + + + + \ No newline at end of file diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 199e7cca..ebe3dea0 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -133,6 +133,16 @@ # ] # } +# Example signalfx-service to report metrics +# "signalfx-service": { +# "class.name": "com.linkedin.kmf.services.SignalFxMetricsReporterService", +# "report.interval.sec": 1, +# "report.metric.dimensions": { +# }, +# "report.signalfx.url": "", +# "report.signalfx.token" : "" +# } + } diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java new file mode 100644 index 00000000..f4d4d6fd --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -0,0 +1,190 @@ +/* + * Copyright (C) 2018 SignalFx, Inc. Licensed under the Apache 2 License. + */ +package com.linkedin.kmf.services; + +import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; + +import java.net.URL; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.MetricRegistry; +import com.linkedin.kmf.common.MbeanAttributeValue; +import com.linkedin.kmf.services.configs.SignalFxMetricsReporterServiceConfig; +import com.signalfx.codahale.metrics.SettableDoubleGauge; +import com.signalfx.codahale.reporter.MetricMetadata; +import com.signalfx.codahale.reporter.SignalFxReporter; +import com.signalfx.endpoint.SignalFxEndpoint; + +public class SignalFxMetricsReporterService implements Service { + private static final Logger LOG = LoggerFactory.getLogger(SignalFxMetricsReporterService.class); + + private final String _name; + private final List _metricNames; + private final int _reportIntervalSec; + private final ScheduledExecutorService _executor; + private final MetricRegistry _metricRegistry; + private final SignalFxReporter _signalfxReporter; + private final String _signalfxUrl; + private final String _signalfxToken; + + private MetricMetadata _metricMetadata; + private Map _metricMap; + private Map _dimensionsMap; + + public SignalFxMetricsReporterService(Map props, String name) throws Exception { + SignalFxMetricsReporterServiceConfig config = new SignalFxMetricsReporterServiceConfig(props); + + _name = name; + _metricNames = config.getList(SignalFxMetricsReporterServiceConfig.REPORT_METRICS_CONFIG); + _reportIntervalSec = config.getInt(SignalFxMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG); + _signalfxUrl = config.getString(SignalFxMetricsReporterServiceConfig.REPORT_SIGNALFX_URL); + _signalfxToken = config.getString(SignalFxMetricsReporterServiceConfig.SIGNALFX_TOKEN); + + if (StringUtils.isEmpty(_signalfxToken)) { + throw new IllegalArgumentException("SignalFx token is not configured"); + } + + _executor = Executors.newSingleThreadScheduledExecutor(); + _metricRegistry = new MetricRegistry(); + _metricMap = new HashMap(); + _dimensionsMap = new HashMap(); + if (props.containsKey(SignalFxMetricsReporterServiceConfig.SIGNALFX_METRIC_DIMENSION)) { + _dimensionsMap = (Map) props.get(SignalFxMetricsReporterServiceConfig.SIGNALFX_METRIC_DIMENSION); + } + + SignalFxReporter.Builder sfxReportBuilder = new SignalFxReporter.Builder( + _metricRegistry, + _signalfxToken + ); + if (!StringUtils.isEmpty(_signalfxUrl)) { + sfxReportBuilder.setEndpoint(getSignalFxEndpoint(_signalfxUrl)); + } + _signalfxReporter = sfxReportBuilder.build(); + + _metricMetadata = _signalfxReporter.getMetricMetadata(); + } + + @Override + public synchronized void start() { + _signalfxReporter.start(_reportIntervalSec, TimeUnit.SECONDS); + _executor.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + captureMetrics(); + } catch (Exception e) { + LOG.error(_name + "/SignalFxMetricsReporterService failed to report metrics", e); + } + } + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); + LOG.info("{}/SignalFxMetricsReporterService started", _name); + } + + @Override + public synchronized void stop() { + _executor.shutdown(); + _signalfxReporter.stop(); + LOG.info("{}/SignalFxMetricsReporterService stopped", _name); + } + + @Override + public boolean isRunning() { + return !_executor.isShutdown(); + } + + @Override + public void awaitShutdown() { + try { + _executor.awaitTermination(5, TimeUnit.MINUTES); + } catch (InterruptedException e) { + LOG.info("Thread interrupted when waiting for {}/SignalFxMetricsReporterService to shutdown", _name); + } + LOG.info("{}/SignalFxMetricsReporterService shutdown completed", _name); + } + + private SignalFxEndpoint getSignalFxEndpoint(String urlStr) throws Exception { + URL url = new URL(urlStr); + return new SignalFxEndpoint(url.getProtocol(), url.getHost(), url.getPort()); + } + + private String generateSignalFxMetricName(String bean, String attribute) { + String service = bean.split(":")[1]; + String serviceType = service.split(",")[1].split("=")[1]; + return String.format("%s.%s", serviceType, attribute); + } + + private void captureMetrics() { + for (String metricName : _metricNames) { + int index = metricName.lastIndexOf(':'); + String mbeanExpr = metricName.substring(0, index); + String attributeExpr = metricName.substring(index + 1); + + List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + + for (final MbeanAttributeValue attributeValue : attributeValues) { + String metric = attributeValue.toString(); + String key = metric.substring(0, metric.lastIndexOf("=")); + String[] parts = key.split(","); + if (parts.length < 2) { + continue; + } + parts = parts[0].split("="); + if (parts.length < 2 || !parts[1].contains("cluster-monitor")) { + continue; + } + setMetricValue(attributeValue); + } + } + } + + private void setMetricValue(MbeanAttributeValue attributeValue) { + String key = attributeValue.mbean() + attributeValue.attribute(); + SettableDoubleGauge metric = _metricMap.get(key); + if (metric == null) { + metric = createMetric(attributeValue); + _metricMap.put(key, metric); + } + metric.setValue(attributeValue.value()); + } + + private SettableDoubleGauge createMetric(MbeanAttributeValue attributeValue) { + String signalFxMetricName = generateSignalFxMetricName(attributeValue.mbean(), attributeValue.attribute()); + SettableDoubleGauge gauge = null; + + if (signalFxMetricName.contains("partition")) { + gauge = createPartitionMetric(signalFxMetricName); + } else { + gauge = _metricMetadata.forMetric(new SettableDoubleGauge()) + .withMetricName(signalFxMetricName).metric(); + } + LOG.info("Creating metric : {}", signalFxMetricName); + + for (Map.Entry entry : _dimensionsMap.entrySet()) { + _metricMetadata.forMetric(gauge).withDimension(entry.getKey(), entry.getValue()); + } + _metricMetadata.forMetric(gauge).register(_metricRegistry); + + return gauge; + } + + private SettableDoubleGauge createPartitionMetric(String signalFxMetricName) { + int divider = signalFxMetricName.lastIndexOf('-'); + String partitionNumber = signalFxMetricName.substring(divider + 1); + signalFxMetricName = signalFxMetricName.substring(0, divider); + SettableDoubleGauge gauge = _metricMetadata.forMetric(new SettableDoubleGauge()) + .withMetricName(signalFxMetricName).metric(); + _metricMetadata.forMetric(gauge).withDimension("partition", partitionNumber); + return gauge; + } +} + diff --git a/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java new file mode 100644 index 00000000..55637e9a --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2018 SignalFx, Inc. Licensed under the Apache 2 License. + */ +package com.linkedin.kmf.services.configs; + +import java.util.Arrays; +import java.util.Map; + +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; + +/** + * key/value pair used for configuring SignalFxMetricsReporterService + * + */ +public class SignalFxMetricsReporterServiceConfig extends AbstractConfig { + private static final ConfigDef CONFIG; + + public static final String REPORT_METRICS_CONFIG = "report.metrics.list"; + public static final String REPORT_METRICS_DOC = CommonServiceConfig.REPORT_METRICS_DOC; + + public static final String REPORT_INTERVAL_SEC_CONFIG = CommonServiceConfig.REPORT_INTERVAL_SEC_CONFIG; + public static final String REPORT_INTERVAL_SEC_DOC = CommonServiceConfig.REPORT_INTERVAL_SEC_DOC; + + public static final String REPORT_SIGNALFX_URL = "report.signalfx.url"; + public static final String REPORT_SIGNALFX_URL_DOC = "The url of signalfx server which SignalFxMetricsReporterService will report the metrics values."; + + public static final String SIGNALFX_METRIC_DIMENSION = "report.metric.dimensions"; + public static final String SIGNALFX_METRIC_DIMENSION_DOC = "Dimensions added to each metric. Example: {\"key1:value1\", \"key2:value2\"} "; + + public static final String SIGNALFX_TOKEN = "report.signalfx.token"; + public static final String SIGNALFX_TOKEN_DOC = "SignalFx access token"; + + static { + CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, + ConfigDef.Type.LIST, + Arrays.asList("kmf.services:*:*"), + ConfigDef.Importance.MEDIUM, + REPORT_METRICS_DOC) + .define(REPORT_INTERVAL_SEC_CONFIG, + ConfigDef.Type.INT, + 1, + ConfigDef.Importance.LOW, + REPORT_INTERVAL_SEC_DOC) + .define(REPORT_SIGNALFX_URL, + ConfigDef.Type.STRING, + "", + ConfigDef.Importance.LOW, + REPORT_SIGNALFX_URL_DOC) + .define(SIGNALFX_TOKEN, + ConfigDef.Type.STRING, + "", + ConfigDef.Importance.HIGH, + SIGNALFX_TOKEN_DOC); + } + + public SignalFxMetricsReporterServiceConfig(Map props) { + super(CONFIG, props); + } +} + From 5dc6395e4882240ce12623e27f612cfaa0990896 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 6 Sep 2018 15:34:01 -0700 Subject: [PATCH 031/192] Fix wiki link in README --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index e3a04871..595c0f9d 100644 --- a/README.md +++ b/README.md @@ -140,7 +140,7 @@ attribute-name of the JMX metric in the query above. - [Motivation](https://github.com/linkedin/kafka-monitor/wiki/Motivation) - [Design Overview](https://github.com/linkedin/kafka-monitor/wiki/Design-Overview) -- [Service Design](https://github.com/linkedin/kafka-monitor/wiki/Service-Design) +- [Service and App Overview](https://github.com/linkedin/kafka-monitor/wiki) - [Future Work](https://github.com/linkedin/kafka-monitor/wiki/Future-Work) From 36c5f2bd84d39f0674ebf1e17402fbfc0c552d98 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 5 Nov 2018 12:58:27 -0800 Subject: [PATCH 032/192] Add contributing license (#140) --- CONTRIBUTING.md | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 CONTRIBUTING.md diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 00000000..771c1929 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,21 @@ +Contribution Agreement +====================== + +As a contributor, you represent that the code you submit is your +original work or that of your employer (in which case you represent you +have the right to bind your employer). By submitting code, you (and, if +applicable, your employer) are licensing the submitted code to LinkedIn +and the open source community subject to the Apache 2.0 license. + +Responsible Disclosure of Security Vulnerabilities +================================================== + +Please do not file reports on Github for security issues. +Please review the guidelines on at +https://www.linkedin.com/help/linkedin/answer/62924/security-vulnerabilities?lang=en + +Tips for Getting Your Pull Request Accepted +=========================================== + +1. Make sure all new features are tested and the tests pass. +2. Bug fixes must include a test case demonstrating the error that it fixes. From 4ce1c51984fb96d0efc27d4d97e8a91d132eadb3 Mon Sep 17 00:00:00 2001 From: Xiongqi Wu Date: Fri, 30 Nov 2018 13:58:38 -0800 Subject: [PATCH 033/192] Add 99.99th produce delay and records delay (#141) * Add 99.99th produce delay and records delay --- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 3 ++- src/main/java/com/linkedin/kmf/services/ProduceService.java | 3 ++- webapp/index.html | 5 +++++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 689fb79b..f55193e5 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -250,7 +250,8 @@ public ConsumeMetrics(final Metrics metrics, final Map tags) { int sizeInBytes = 4 * bucketNum; _recordsDelay.add(new Percentiles(sizeInBytes, _latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, new Percentile(new MetricName("records-delay-ms-99th", METRIC_GROUP_NAME, "The 99th percentile latency of records from producer to consumer", tags), 99.0), - new Percentile(new MetricName("records-delay-ms-999th", METRIC_GROUP_NAME, "The 999th percentile latency of records from producer to consumer", tags), 99.9))); + new Percentile(new MetricName("records-delay-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile latency of records from producer to consumer", tags), 99.9), + new Percentile(new MetricName("records-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of records from producer to consumer", tags), 99.99))); metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), new Measurable() { diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 58eeca20..57dfbc1b 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -251,7 +251,8 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { int sizeInBytes = 4 * bucketNum; _produceDelay.add(new Percentiles(sizeInBytes, _latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, new Percentile(new MetricName("produce-delay-ms-99th", METRIC_GROUP_NAME, "The 99th percentile delay in ms for produce request", tags), 99.0), - new Percentile(new MetricName("produce-delay-ms-999th", METRIC_GROUP_NAME, "The 999th percentile delay in ms for produce request", tags), 99.9))); + new Percentile(new MetricName("produce-delay-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile delay in ms for produce request", tags), 99.9), + new Percentile(new MetricName("produce-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile delay in ms for produce request", tags), 99.99))); metrics.addMetric(new MetricName("produce-availability-avg", METRIC_GROUP_NAME, "The average produce availability", tags), new Measurable() { diff --git a/webapp/index.html b/webapp/index.html index c27cfaa2..5c2a86d4 100644 --- a/webapp/index.html +++ b/webapp/index.html @@ -186,11 +186,16 @@

Kafka Monitor GUI

var monitoredMbeans = [ {name: "kmf.services:type=produce-service,name=*", attribute: "produce-availability-avg"}, {name: "kmf.services:type=produce-service,name=*", attribute: "records-produced-rate"}, + {name: "kmf.services:type=produce-service,name=*", attribute: "produce-delay-ms-avg"}, + {name: "kmf.services:type=produce-service,name=*", attribute: "produce-delay-ms-99th"}, + {name: "kmf.services:type=produce-service,name=*", attribute: "produce-delay-ms-999th"}, + {name: "kmf.services:type=produce-service,name=*", attribute: "produce-delay-ms-9999th"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-lost-rate"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-duplicated-rate"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-delay-ms-avg"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-delay-ms-99th"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-delay-ms-999th"}, + {name: "kmf.services:type=consume-service,name=*", attribute: "records-delay-ms-9999th"}, {name: "kmf.services:type=consume-service,name=*", attribute: "records-delay-ms-max"}, ]; From a0bb375c5af8a7924c2e628d1be01434b196159c Mon Sep 17 00:00:00 2001 From: Xiongqi Wu Date: Fri, 22 Mar 2019 20:12:23 -0700 Subject: [PATCH 034/192] fix incorrected calculation of availability and separate PLE routing from rebalance (#151) * fix in-corrected calculation of availability and separate preferred leader election from partition reassignment fix two issues: 1) Incorrectly availability for offline partition when setting _treatZeroThroughputAsUnavailable to false and large producer timeout. The sample history KMF keeps is 60 seconds, which means all samples expire after 60 seconds. If We have request timeout more than 60 seconds, there is some time window the reported availability will be 100% even some partitions are offline. 2) preferred leader election is triggered while partition reassignment is still in progress. This leads to potential issues a) the replica might be moved out of the selected PLE leader as a result of partition reassignment , which leads to multiple leadership movement. 2) the selected PLE leader might not be the preferred leader after reassignment. --- config/kafka-monitor.properties | 1 + .../MultiClusterTopicManagementService.java | 70 +++++++++++++++++-- .../linkedin/kmf/services/ProduceService.java | 13 +++- ...tiClusterTopicManagementServiceConfig.java | 11 ++- 4 files changed, 89 insertions(+), 6 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index ebe3dea0..cc2e13dd 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -51,6 +51,7 @@ "topic-management.replicationFactor" : 1, "topic-management.partitionsToBrokersRatio" : 2.0, "topic-management.rebalance.interval.ms" : 600000, + "topic-management.preferred.leader.election.check.interval.ms" : 300000, "topic-management.topicFactory.props": { }, "topic-management.topic.props": { diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 0882e368..90ac42c8 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -74,6 +74,7 @@ public class MultiClusterTopicManagementService implements Service { private final String _serviceName; private final Map _topicManagementByCluster; private final int _scheduleIntervalMs; + private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; public MultiClusterTopicManagementService(Map props, String serviceName) throws Exception { @@ -84,6 +85,7 @@ public MultiClusterTopicManagementService(Map props, String serv ? (Map) props.get(MultiClusterTopicManagementServiceConfig.PROPS_PER_CLUSTER_CONFIG) : new HashMap<>(); _topicManagementByCluster = initializeTopicManagementHelper(propsByCluster, topic); _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); + _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); _executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { @Override public Thread newThread(Runnable r) { @@ -109,8 +111,11 @@ private Map initializeTopicManagementHelper(Map entry : _topicManagementByCluster.entrySet()) { + String clusterName = entry.getKey(); + TopicManagementHelper helper = entry.getValue(); + try { + helper.maybeElectLeader(); + } catch (IOException | ZkNodeExistsException | AdminOperationException e) { + LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); + } + } + } catch (Throwable t) { + // Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime + // and such error is not caught by compilation + LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); + stop(); + } + } + } + static class TopicManagementHelper { private final boolean _topicCreationEnabled; @@ -189,6 +218,7 @@ static class TopicManagementHelper { private final int _minPartitionNum; private final TopicFactory _topicFactory; private final Properties _topicProperties; + private boolean _preferredLeaderElectionRequested; TopicManagementHelper(Map props) throws Exception { TopicManagementServiceConfig config = new TopicManagementServiceConfig(props); @@ -198,6 +228,7 @@ static class TopicManagementHelper { _replicationFactor = config.getInt(TopicManagementServiceConfig.TOPIC_REPLICATION_FACTOR_CONFIG); _minPartitionsToBrokersRatio = config.getDouble(TopicManagementServiceConfig.PARTITIONS_TO_BROKERS_RATIO_CONFIG); _minPartitionNum = config.getInt(TopicManagementServiceConfig.MIN_PARTITION_NUM_CONFIG); + _preferredLeaderElectionRequested = false; String topicFactoryClassName = config.getString(TopicManagementServiceConfig.TOPIC_FACTORY_CLASS_CONFIG); _topicProperties = new Properties(); @@ -251,7 +282,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { try { List partitionInfoList = getPartitionInfo(zkClient, _topic); Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkClient.getAllBrokersInCluster()); - + boolean partitionReassigned = false; if (partitionInfoList.size() == 0) throw new IllegalStateException("Topic " + _topic + " does not exist in cluster " + _zkConnect); @@ -266,6 +297,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}" + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, expectedReplicationFactor); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + partitionReassigned = true; } // Update the properties of the monitor topic if any config is different from the user-specified config @@ -287,12 +319,42 @@ void maybeReassignPartitionAndElectLeader() throws Exception { !zkClient.reassignPartitionsInProgress()) { LOG.info("MultiClusterTopicManagementService will reassign partitions of the topic {} in cluster {}", _topic, _zkConnect); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + partitionReassigned = true; } if (partitionInfoList.size() >= brokers.size() && someBrokerNotElectedLeader(partitionInfoList, brokers)) { - LOG.info("MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in cluster {}", _topic, _zkConnect); + if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { + LOG.info( + "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in cluster {}", + _topic, _zkConnect); + triggerPreferredLeaderElection(zkClient, partitionInfoList); + _preferredLeaderElectionRequested = false; + } else { + _preferredLeaderElectionRequested = true; + } + } + } finally { + zkClient.close(); + } + } + + void maybeElectLeader() throws Exception { + if (!_preferredLeaderElectionRequested) { + return; + } + + KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, + ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); + + try { + if (!zkClient.reassignPartitionsInProgress()) { + List partitionInfoList = getPartitionInfo(zkClient, _topic); + LOG.info( + "MultiClusterTopicManagementService will trigger requested preferred leader election for the topic {} in cluster {}", + _topic, _zkConnect); triggerPreferredLeaderElection(zkClient, partitionInfoList); + _preferredLeaderElectionRequested = false; } } finally { zkClient.close(); diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 57dfbc1b..573f185c 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -225,6 +225,7 @@ private class ProduceMetrics { private final Sensor _produceDelay; private final ConcurrentMap _recordsProducedPerPartition; private final ConcurrentMap _produceErrorPerPartition; + private final ConcurrentMap _produceErrorInLastSendPerPartition; private final Map _tags; public ProduceMetrics(final Metrics metrics, final Map tags) { @@ -233,6 +234,7 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { _recordsProducedPerPartition = new ConcurrentHashMap<>(); _produceErrorPerPartition = new ConcurrentHashMap<>(); + _produceErrorInLastSendPerPartition = new ConcurrentHashMap<>(); _recordsProduced = metrics.sensor("records-produced"); _recordsProduced.add(new MetricName("records-produced-rate", METRIC_GROUP_NAME, "The average number of records per second that are produced", tags), new Rate()); @@ -277,7 +279,14 @@ public double measure(MetricConfig config, long now) { // will block and retry for a certain amount of time based on its configuration (e.g. retries, retry.backoff.ms). // Note that if it takes a long time for messages to be retries and sent, the latency in the ConsumeService // will increase and it will reduce ConsumeAvailability if the latency exceeds consume.latency.sla.ms - availabilitySum += 1.0; + // If timeout is set to more than 60 seconds (the current samples window duration), + // the error sample might be expired before the next error can be produced. + // In order to detect offline partition with high producer timeout config, the error status during last + // send is also checked before declaring 1.0 availability for the partition. + Boolean lastSendError = _produceErrorInLastSendPerPartition.get(partition); + if (lastSendError == null || !lastSendError) { + availabilitySum += 1.0; + } } } // Assign equal weight to per-partition availability when calculating overall availability @@ -322,6 +331,7 @@ public void run() { _sensors._produceDelay.record(System.currentTimeMillis() - currMs); _sensors._recordsProduced.record(); _sensors._recordsProducedPerPartition.get(_partition).record(); + _sensors._produceErrorInLastSendPerPartition.put(_partition, false); if (nextIndex == -1 && _sync) { nextIndex = metadata.offset(); } else { @@ -331,6 +341,7 @@ public void run() { } catch (Exception e) { _sensors._produceError.record(); _sensors._produceErrorPerPartition.get(_partition).record(); + _sensors._produceErrorInLastSendPerPartition.put(_partition, true); LOG.warn(_name + " failed to send message", e); } } diff --git a/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java index 5ac0f61a..337f86f8 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java @@ -29,6 +29,10 @@ public class MultiClusterTopicManagementServiceConfig extends AbstractConfig { public static final String REBALANCE_INTERVAL_MS_DOC = "The gap in ms between the times the cluster balance on the " + "monitor topic is checked. Set this to a large value to disable automatic topic rebalance."; + public static final String PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG = "topic-management.preferred.leader.election.check.interval.ms"; + public static final String PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_DOC = "The gap in ms between the times to check if preferred leader election" + + " can be performed when requested during rebalance"; + static { CONFIG = new ConfigDef() .define(TOPIC_CONFIG, @@ -40,7 +44,12 @@ public class MultiClusterTopicManagementServiceConfig extends AbstractConfig { 1000 * 60 * 10, atLeast(10), ConfigDef.Importance.LOW, - REBALANCE_INTERVAL_MS_DOC); + REBALANCE_INTERVAL_MS_DOC) + .define(PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG, + ConfigDef.Type.LONG, + 1000 * 60 * 5, + atLeast(5), + ConfigDef.Importance.LOW, PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_DOC); } public MultiClusterTopicManagementServiceConfig(Map props) { From 100eeb795af89a7f70a267097112799e51e0f959 Mon Sep 17 00:00:00 2001 From: Xiongqi Wu Date: Mon, 15 Apr 2019 13:02:16 -0700 Subject: [PATCH 035/192] add bintray suppport and upgrade gradle (#153) add bintray suppport and upgrade gradle --- build.gradle | 94 ++++++++++++++++-- gradle/wrapper/gradle-wrapper.jar | Bin 53637 -> 54227 bytes gradle/wrapper/gradle-wrapper.properties | 4 +- gradlew | 68 +++++++------ gradlew.bat | 14 +-- semantic-build-versioning.gradle | 0 settings.gradle | 16 +++ .../linkedin/kmf/consumer/KMBaseConsumer.java | 2 +- .../kmf/topicfactory/TopicFactory.java | 2 +- 9 files changed, 148 insertions(+), 52 deletions(-) create mode 100644 semantic-build-versioning.gradle diff --git a/build.gradle b/build.gradle index e896702d..0e947120 100644 --- a/build.gradle +++ b/build.gradle @@ -1,14 +1,27 @@ + +buildscript { + repositories { + jcenter() + } + dependencies { + classpath "com.jfrog.bintray.gradle:gradle-bintray-plugin:1.8.4" + } +} + +apply plugin: 'maven-publish' +apply plugin: 'com.jfrog.bintray' + allprojects { apply plugin: 'idea' apply plugin: 'eclipse' apply plugin: 'java' apply plugin: 'checkstyle' - version = "1.0.0" - sourceCompatibility = 8 targetCompatibility = 8 + group = 'com.linkedin.kmf' + repositories { mavenCentral() } @@ -32,22 +45,63 @@ allprojects { } tasks.create(name: "copyDependantLibs", type: Copy) { - from (configurations.testRuntime) { + from(configurations.testRuntime) { include('slf4j-log4j12*') } - from (configurations.runtime) { - } + from(configurations.runtime) {} into "build/dependant-libs" duplicatesStrategy 'exclude' } jar { + doFirst { + manifest { + //embed version information into jar manifests + attributes('Name': "${project.name}", + 'Specification-Title': "${project.name}", + 'Specification-Version': "${project.version}", + 'Specification-Vendor': "LinkedIn", + 'Implementation-Title': "${project.name}", + 'Implementation-Version': "${project.version}", + 'Implementation-Vendor': "LinkedIn") + } + } + dependsOn 'copyDependantLibs' } + task sourceJar(type: Jar) { + from sourceSets.main.allJava + classifier "sources" + } + + task javadocJar(type: Jar) { + from javadoc + classifier = 'javadoc' + } + task testJar(type: Jar) { - classifier = 'test' - from sourceSets.test.output + from sourceSets.test.allJava + classifier = 'tests' + } + + publishing { + publications { + MyPublication(MavenPublication) { + groupId project.group + artifactId project.name + version project.version + + from components.java + artifact sourceJar + artifact javadocJar + artifact testJar + + pom { + description = 'kafka monitor' + } + } + } } artifacts { @@ -56,7 +110,7 @@ allprojects { checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") - configProperties = [ "suppressionFile" : new File(rootDir, "checkstyle/suppressions.xml") ] + configProperties = ["suppressionFile": new File(rootDir, "checkstyle/suppressions.xml")] } test.dependsOn('checkstyleMain', 'checkstyleTest') @@ -71,6 +125,26 @@ allprojects { } } -task wrapper(type: Wrapper) { - gradleVersion = '4.8.1' +bintray { + user = System.getenv('BINTRAY_USER') + key = System.getenv('BINTRAY_KEY') + + publications = ['MyPublication'] + + pkg { + repo = 'maven' + name = 'kafka-monitor' + userOrg = 'linkedin' + licenses = ['Apache-2.0'] //this is validated by bintray and must be a valid license + vcsUrl = 'https://github.com/linkedin/kafka-monitor.git' + version { + name = project.version + } + publish = true + } +} + +wrapper { + gradleVersion = '5.2.1' + distributionType = Wrapper.DistributionType.ALL } diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index 05ef575b0cd0173fc735f2857ce4bd594ce4f6bd..51288f9c2f05faf8d42e1a751a387ca7923882c3 100644 GIT binary patch delta 24335 zcmY(qV~{4W7A@MGY1_7qY1_7K+x*(LZQC}dZM(m=t?BN0bIyHrZ=I^_+LcuDE2(6y zwO2xm!I#Rx5fo*>Az(m2prAmiyxpY|5y;{HlR35A!h}FTK-3cjm9Z|d&#@nz@0~$G z{@>(+|4gnRg7|Nqru;wol@a+r{mt)i&~YdbkQ7u95US)v2#jQTgrCWIqX>XhjYuEV zHRQjwiF96CaKEt{m?J^Jg_D>)I+D2a->#h$6E{+I-vOq{Aj@ci*V40 zM1TmKVY#>yg;JP=I?RECR=5Yu>Ii$* zS7vjv@-Jj{%JTBLiHxM$6B)Iwce%?>q_aX+X#(m*&Iu5O*;(Bz-zvg8yP`7ry)Avm zZ}@=ciT=>lW8Nyj^8>(OVV<@qV)+kwh755bR(d^JX-%ta{iY{fi^(0zM_gKEi6^{` zJxiGjampn!rF`axUm<%rN1XjSZ?PT%~BR+-5>Ppr+!XsCQKE#&k7?)2T}qxacbt*e@ENlB_iIC|sC|6SVh@qvGJ0c{@pH zI}c?lqtOt=Z#$TQHMA}qovR9F5q`$9JFYcBwFYYUYA1ls=apK{W)8k0wpsmLXSdj- zb+R?JN3+CrH*E>CK}ADlE{&3uMJ3)C8HB=C!c59snAP*;D9MCcPU;eg*L9MPk_wa# zw!zTHUpAaalPpwbhKXT}WvXhsm>es~GxCG`qJYXx;j94-tD(O>_OtSU0)D)!p_ak_gL z*6iNtrxs}k`7XBNeac_rQiBRwvRUguj6aVwVep@@=ATK}a{PWtLF;xY z&RWP+_VNp?{LJjJ*vLz@c$Eg!SQZ9=nSuU62lUT4jsBnHdyBSef2JD;%h9F}muGyP zyh{z|P0nYC6}C+H0E%}KpVT1&v~RGKVi+dGpK619cdtm_h5Pw$=^-NE;;Ll6Pcim# z5`=&S^Xn-_jFuSwN}G*}yjQn#^DIUEntPnIycwMfEd}4K?K?^#?Iclo^ zHd0MaOyEEpYPMg;?0RQ~y~#cgwc1lBO%_l-_7qer&)^N6Z4%6bmmRU!*@~c$J%;@) zkE~=#l;LWw4ZP_K*@OBZ`K{v~>>>a`V_-Z}=earC4u@Z01E>9Cv=znj^)ZT2w2tM0 z<>b$HboHGPk~xJ%LwRjGfNJ6q4qE4sRGe&S#~*R~Fw=nX$wzadu87>%{`D)S!U;fl zd3Fh4{+hHBge4dV7VV9y;I<|GNC?*#b4?G|7jbP5?+ZU0Ch+E4;3M!RCD|^p!apFO zH^10qV&UjZhvM;Y^;DsA@cDF2V?Eok~*^EVQ zf}Z`OFZ_eM9x2HE?vYpVhg!Vh89)!rw1tk?o((|(&~_6@cSTo^q-FcANxqTJ_&Og( z{dm8OLI0d#Q0~7V#CRi++wquZn<^yEphX(S5aXg`r;a(Zf;F7}Nt#Z^%L{~j8*C4? zX6L&2GuLPZqJ(1fUco9si+}@`tdj*H9vy1>lLfByifwgK(f(}zK7tKZE zR}i_9rX-3>yi#;E7C;_nRbGLM?ZYPI6*jrX?WF?QX=D2Lt6m820Knzvjy%4ZD`}Qg zL#D5l`%0TKe82z>Raa~Yg)p*hS!b-T5o*y;O{;xb9eQF}o%Cy^agd^YR%#)@doDL? zLx>fZuk#H(gTuHSjGc zQ50JDv0s|yDInZ94KU&&CnCLRQs{)n2#iY>mI)zqA%P?z%Pwzb$+S~c1hWU>yg!Hi z?X+Kz?0?`X45OI3@}YQFqPD(*6-LcXK#GoELK@QYTU`2sSSv5O3q4W;qxJ&Ad}5GS zK~)j$Cpnec(92g~+JM_~JH$!S8nWzcZErxBXV8P2eAStb0zh*>>%lKg`a>yd)Z}Oo zs41tl9!44YH-$3px{m%oEsnm2JC>Cst~TVYAY!!!w->EdD9svm=j6L!$xa+xw}(T= zUJftsMYLsRW#{B9?YDj!U<0)44gv@W#eWgf9To<_0QVm>Vvy9^!Z!Swzr`tQxHM%E_6HNlg06v zdpn)o{`Y(C@-LVp!rzz!-WZXTx^cw;@t7p)VR-C34W*Q8IW0{ZY331t;ct1Z63n6o zGk0UfgGqO>uxuLpQAa!-X4@&i>0KigKfy5N-2@hYD5ls)OVr%&&kZbjpbLwy{-_a_(b3f2|0S|Aa`dB;6jEmnGnRb3 zp+4o8*f57l8cTe(STF@id{)}IzVdt?o~}Al7S(a3yX$64Oy(ruB+H2L!bVKw6ZXw) z=a(72ZP+$^a>bu=8Fyk}_vCidv-3s+t4mim^H{~NN~@)X#v}i=!)wby$JfWkEL*cn zYJhpm+Klk%s+foC9epoCPo(34tzSJlB>hZ_YXwnP8P8Y@gQ)l;+hsJqd*uj|EueIg zP#1Dht+~eV*ZT?JsyCQZ$5g*|eU!GVStt_kV^M^@r-j$obIB2}*HAVKa@Yy|>0LUm zQNK~ngC}|G56APc%~qYN#a62mB#a4fVHLDPS7o`3$PAD*F`u|R~kGu%%JG_P;4;{S4q^e?!9;d#VoqbW$3n#i6(pfnU382`gu+CXTk!Kk7|m|Ag{j z2Tg@{Cd>aRqrZVM91{nXlNg_@u^gD{P_EX-A`%S`)Zr3Ig5|? z9vE8%Xnu!)PR!eHuUB+d#L+BKp1ow?3rSuZgES>01c@MP{+^9AZ8K|9uKn;dv2Yq@ z*hxxXi0%!J=LSyT>NKy7nnpWHP7s$jx%b=T$(rl@NhKge3%eHdm<4QmAp&8F^zFX2i^}10GH8rGO#2|4McQ?Cy|H1 z1WJWNyr91FN|XEC4W-Be$%!bKu-88s`6_c9$`{pI){rJk!>pm^O2U@dMH#}}3+sLZ z3>I(@u1VUIMSf~%qlW_}I9J9x;OBeh zI9Ov&X-@<>V^$*f2Qr<{+13#`VgkR69d&!1>!&R`c>MFO5NB_){cY{x4!r)MKAQrI z%2G=XDX=;3ZwR36rVy6!nYUtDb5O1R_xm~0TT>;yyZ7&LkxpW%!GThKZoEzG3s^pl z(P8*_N^EQ_nG7e(n3CjmV9V}rfV9Gn4Pu@Vo~;vSpjy2#n?A`sqt=0k)c#-cJXCRt zrF&EK;CiC0uIRR0SKi}V?$R^hxsM@3m5ax8pIuAg54Ou^Y;V6b zE=@DHGoQL^Q~YzX)gZVUpVP`#3T6V+a9_F zH4nzA$8yzd0hK}7>RwGX4~%!eN2AsB(fZeNLu^M5AJX>z(C;~1e;@A=)`P4^Jv!M6 zYLKrM>(DD$uVYF;@Zop^BMN2oUI5i929O99Uj;=^)yE0i2-U|CVdXTAs`&Avjc^L% zM#h37ePE%S>NGE@2~K$9hB@M4hYgNJV8hb+dVz^FoQ^=OQC77(#pQh)Dl1Ov2D-J> z4VN0y>@Bw%3xI^gOM#%?O^DZ`>_!=D#>G9~6t*pT#cYTC0tbHoT#Jly0bsFRyhP`L zCmM`|R+05$(|z^6)MK#)oq5%qE3IFIC~Mk0`vZamyIpW_Gw1ksZ)qav`EwR{o9fGU zkJW2tUg?d>NQ8i?k_^qMcW|Z99Njt#`$=YT@uJK%cWS$g*k#F8gY8D%+>~+SlIyh4 zIkM$692D5~#604Z9ChF79YFnoo+s5K*LUKv1X#{eW=9)cE7#m}gTqPu$e(KEy79Q> z_tBA^Vap*vH;?4Zt(rr~f-kW-IsmqVrJk77e89}&VnnqzO|nw+MJNb{KB-?}`u=p! zJVna3hHG)byF}eVY6K$OrpDU#GUKB%D_fY$+&d5~>q+eJwVe|a50F1K8(yr~^7hg3 zP~@M*n^4*_UT9GFt`Qo4k|8+e+45e3CG7>KIiurP9Zn9!WB|cgOuEJ1O(&B2W(Jn` z<}r9f-p!!LB8Jtd)Mta3f>SnW9+y7B+Ubiz;+{Lm83j@|TRG}S=%qEsDPqngi=-r2 zvml-}^e;Fei+cV10qxgxTJA#L+_Q4P?H=bqT*Zaqb58ctbW z{e>n56H2vQz|;Xg>~>R3xbf%M9!fa_oZ_;ma;1zqa$v_LcGXY}aK!A)lb+30d{_&= z9ta=EW3R3WW=7!Wi|SzIsWZ;wsWA@p6dv~B-W~r5rq+7^C}G+LhB=>#xVH-7y%TFO zjmb`xPk{zx_Aw7({PN`(NdCppg)Y)F8{o$j)Il+oMKOb3S+!j;y?@etF$VAZzen z_|7PnOnvbYW{Kd*=h*K`oI1-`NOa=~q=>zOE+~4(5u~Q-Sz?k19PQwVs zr{7eLYjR2IPNjvPxKb|EZ$9cN#6S@7@ei};L#eh0AjH{wE~c3)&D=pghA;&O28;W* zaNKzm;z4hRb`v)wwzG^Vm6!dTbF&VBzzd4)%(Q`Z_&TCK)!*_(F7kRtq_VMp13lh@ z9MS=6gjy4?yDab*u(qTSi$ab%AowmM;-qFJ;iMKZSpvl;{VC z91(XUK=ZGdC9k?{K@-EQ8JHSo;jP_)4R=z6Y{3UuTUGA~Z`HEC4`^(%=Bfa>PJ52< zQC)Ug?{$@=gc!FwG;Ny=iC$*KCpLc1vdzq1@l}l14Xkbc$mDlg*GF^*AHJL@rK1FQ zT7(=CvZKLYcA*2cNGi;zl>E++6$68ORc~!`0Iq$3SgA9Z&Be`;*AgC5HlsM;IMUGnAEQ^Za z#Tqj%4PiXXibN)8ay_K^q_Mp1_T2ejke}DT;zq7J0>bae7vErN-+tJ=Y8MO=uO}8#?@0h@q@(gq5u7Tq0}+fOV6I6;l-^x|J`* zo*R4usOK<32sEe5romvoI^Sl(pC(<`9?)A#50;A8=WS4}d;8inKTg%KVANne+i4?I zs$^Ckw4*q{51!F)T|*x9fBR!e?r!ty6bvex6mOi1K#jAuiDL$Xy;W0ULdtDZn}^3FP7CIbDN16 zC=>cIoxx|LP1=hD0C@UD*AcnW0Qq`{G{*Ue!~Y&5Q}5}$q<;s84>}ncnG>L+qN9fP z{Y#3I7;mzdO;V?%HAF|5QIfoc28|3Ij2x;||AIJ&B0S9Ha6$^y;rT<`0MFtg*a2ad zb9OnM>359Zt3>6VJjv1Oq-oR3rj4J^b!&g?n*8_2pLhY#o(KoOv(b_uYQhVq^Iq}XASa{XE`#hX6j+u z_E8z84gdC7w{j2V=pU31A<~y6tre}bqyNzCZdh2rv zL$H<;eQVK1SPo{aJ*AZ?MA#`E53=<7rnoes-Ig|Y?N;Hs{n$b~hFvu3_&jTKmQ?!; z+9qn3$~ihGAytWuW+66dY4}f=r5ckG;YaPP%bu_ale5jVfPOe!<|VE2eG*xw##vH; zsgXBjT{-?R%el+Wk_P}=vKe)Qe#do^OIveq8Hk|dHYOGrm-Iy%XKxcno-B~cKdEIe zC*rjU7ArQw(3~c#PB5I#!1a8N^2&^BbWR2I;rJpvEj|XNtcupP>l_P!XC^hWmR~iG zIk#}}JIv_INNojr2S((TtMXE%S9L^SLTJaoX0EVE-INm583Vv|_mrZc1MA9FwZ(g? zNdik?M8@pvm7D`_fFtQmWhS=gI=i@(k;`Ra6T z^-i4CMk#Kj5|tYvRVYj=C4r=nZBcM|kXq=kh1Kh02#n$<5d{T7dgz2eRP_8qR`h&e zU=-tBX;8hYFCf+d;)lYnG8lbox~3p~X{?<+T0QO7=dVjeFHqCVDHoU=F89=zR8h}E zYFg6Z30GNT39yw{ARj;N5h)t3H5S~=Nr*xcS+*~wVN~rWUaEQdjKP-_DsS_6#;KEZ5mVyfqS#I z-07+VviGnRlJwSUg>!RZFwIG*KitWBb7*TGKS(vBOn|S5&8NgPGJ{+5f7Jew%lK9|LL|( zkKX@hg21~tTyYIyLG-Dz?F3I7tB5_kEzz$<0DHC&2$Bo7kO-cG;TdwN+66JT3OG$5p(j0}n-oj5*jEMdG^V{eh zvVLD9=0r-@xP{*p1^tM@=n2Uqb5q0>bwk42t|uz}j)i)~F_>23oV>Lq>pULU+)H zE|i&Y&t;8ChDnJ|e$e9|ko5&MD1Q;x_jxq<-&-rU8tZNOAE)|-m>dUzlYBe!BUyF~ z3XuDcr=e+J`zyS#OqmBDX*=pzE{Lzw!NV3RLI=V?+lizL;tE5UxHa+E{-9vJyrN65 zdJb!Ev#wSzweD4o_NYuEN3*-E-Ta9D8_@ru@m6um(VV?wn+A2_^^(K2=bQfOyT`M~ z<@IN;zVC-2?sprU&Rqu_VakCB3l@v)1Rz0yBs%B^Yb1ocH5Emos1U}|gDJVA^vM!Q zqgh7cVB(;hF-j>JKOAFVlcaB=Dheklg+(%F_#=r|5h~%JgxG;elW!9$;W#Dty_BzX zm;$Av&`7@2Ve^|VN`AUw@|!T20{LAFFF)xp=IFPKl<;>`m<2m;`2j7Ao&{X@IKcfe z%x{Q>&vK(|C~BJ;7?aq#T~p&h>l!6LNb_ent-PVD@1dqL%As?%!(tAXt1KTMU>hP{ zElPjOO<8L^$||Z*$Z_gDYP-M+)0WA1iCvm>ZbmueRiTyoQn|`xvLyaDF4rU#JM@Mr z!;W+G3JX1l1pe^5MeSyzs-aFV1%OZ5a+{bV8ixDt<6@*7`0e#ownq}jnj!Z*3wk?&tow`Zvg6HvwIA=|7- z;6BIQW`l;d49!wvo5L2agI)B;p4uXh*{&(VvpXy2w1O6jZ|xaUD@vzS6lW6 zroCysA&(=2LmHX6*04rr=!bo94PhR)*$Me#6La+$mj7t+nqM?})?vcnd(>%WTt2?> zH-uz&gNgmKJ%F0^ka!ZXLQY#H@^N7YrsDDtJGlJ9s6b& z0iVaIKkn5qJx<6sxGziO{EdP@RdY+a_zQiCAtNs?+EZw}{mxF^DcVPS+{2VF0@KYG zb8qS158I!pDgB>1DX@6Y!azOJmI61oc5mYKh5DJIbopHJ2!D8Q0l@LE17g0F#u*$m zg=x5c;`A>#K{j(&byOM2kF)nw9dyO*Mdj}WF6;BzZk931OVUuRKF==g*m&v<<8u4Q z3s}At#|a$FP49O+hN#g*GDJ2EdQ`qqd=~-hJ!QtX4j%$+{dbhPw=xp$enJoX}aAi0vzun!|pj`!)151bErr@@5%@9g=Pibk8oZ zGZ^FEY@^herN;n~lVXC~KCCXa4PG_8{>Wgtd zt!7!T*bI*77_&qW^H)F4`>qH14-;Hx@V#UaMz05NXojZ-qk+Vs=LrmVm8LuX1W;oE zyY7i5&7*!_adfh}=SvZ%(s)IzW=~h_Rjj5^Fp|l)(Ho&%4}rehAONE`#L^UTZGmf> zzw3rX9>9ogG2@#PXi1|zx(R*HAMUZN+O48`R(*&-qvmAs=A7H1AXabW2T^Hxc3l_A zUBk7_htzZY&Wj{7T6d5eg&D{9@Wc?xk$@G^4$&Ddr0jG2gu^HdUPau;YC0?nYgS^>+bO+M(H7} z6R-H;*>Js(%E@1o7|fY9;u&OrbB$ z;;9LLP*Amx>;@XX{D`H-c-Y_#9si>ybA;u?3j~o_J|!*to+uW_O3D(Z z0~#Vn&;*LT5N)*29ik~^l7(ysv+EV{ZZo|v4Kq7U#rtCuupl7JqtS%o;Fy);lD^TO z$wT`0>+{tDM1N3$Mc7kk%xGFhGr>`4jEG@UbA%i_J@IfV363z$Om|SNuLpoS0#jZR z+(B{B6+(@@qnYX`GS>ZDvZM3omu9jfZNzIw=a#QmKBGsFqFD_oEionpaJ(l*gi3N9 z35l~9JByEU05i501}WJ*4*E>q>W2;k=31tkoZea_igc!ynt>X9?4FJ2!tT5@m@)Md z4H-;Y){@p?V5z-53$4>G+ch9bE}dMBb+OHQV>%0cpYgsppm?EmS?{?oYhJB5L2@h2 z9L-!W%A5j&Eq8|dzCt~a_o)>bgQ%NtX-mCd%zjMA-DRXWo; z4w?DN8f#i!*Fb8evBNpB$Xyc5`_h+(>?Z3mJ)_s?1OjdaT9dB6xdxEy=tai0p3!kF z$LRGnV?HN4?KJhMqt#|*Q>K(uphW$Ul_*nbM%q0{_jB5@g<4_e_h`?Fszhr|v8iEg zXgzB%)=*IZLM(#wCf&bP`<9_w=nS*;RGdLdkgPsZIPM0#hu!%oqEGK{RcZAHPLK=B2jted4}v}3WE}Fnu2o- z?~@;TRUsh?XueT%6><7FNnKjTJy0+4Q=q2}=AdYOJ|trL%Y#o(}#e?`%UOt@VQV^#x1w&P7&XzD=5kV|t3SSTc;Xay*_B40j7 zRMGLL6b_~$0D{wRWwdu1NRk49szMD>v)K30M-7^Tc@0>~H0wM-#6Kc>WIMU{&xMqF zLbtL+cmSjbl*u=W^IZ>8y6@0$3Ef)`I4I96h`C~(FnR6p-+w^CoRN~Bjs+r(bO)f+ zkq7O-rXq%m`_IHRqx149n2(7jO>mrqQ3mj(MRIK8Ng{zyI^twDlxKg{glFvXMr_F3 zs?uQ7U%BshiJ~M^mP|wYh+0$BCrkofrWBYnG68Vc)y4qGgVIV1mp+xl0M9dv7+LrT zO3xoOSTOh%`}?F2L*E93QA3P>zaP=USqhC95fzw}#g683hv1TQ@&cehvnV9T&wZna zoGM(7egzTa5HtC^BEQNNxfP_d?G0&xtt+GoNKht*w8+GW5oX-t!(FGj~WXe1Y{Q*1cdni*UAt8xth?vXls7oc`e}G3umN_tqACtgyfB7rW|~@a%RdtZ=*S7Jpgc zpIX|j=_ewM+g~R9>8t-GcX(py^2EPL@xHsYLa-QiqG7n#0|fNm0;U0P^YMFM`XD@` zDS@1M)pg`sxmF7egE-55%}Wd-3|n z!yTp$c%0wr*nottpT&X1z6#^7Zl8j~i11gVD%{^v{C~BQKBIHKsv(Z|;PA%!FW(Ts z|3>~@yx+s|Cu{$f82@|m))=N?`4&k`6vBW*rN(AWQV?0dAlx<2R(M%7G2mo$W*o{O zJBdyUu!^)?9i`E0IWul$80yk2j&AMJq=am-HtwbPWl2H;0?L{Q)k_YAEvX zH7o#J-H3jw3bePiLxr%+PRGbmXaB(WDDxQuw3e~Bze27}9^1D2DB|Cs&UX;Bo!DkE zdCqb>xLMR($$AWVENx&D;PU1*=rLZKQab~twP)0l(yO>T1j18eE98ooc8zedH!5Uz-t zif+MUzbduzX_%pGt!Q!5Tfc7Z8$BnMTbB3|X4fZWXR@-K?c%$#ZKNVUQm)_6ZKd2@ z=ouR?`!kwc9ulVMqq%`vd+yEZVPi)-(mn> zQ(0{Urd8|VVy#?u;)IK75`T(MdtAp?TDFu-m7ALCaf2q=ww@SP5;F)RT`sVdVomlV%IC$Ch;SWa|LFq^FlX{Mvyklfs)V=FNy zB^rckT8Y%41JBXT#Usy^?C@>CGF>QnPEU4|SzAL5J%y9(B(oFaiJU2u1qtl9s5}BD zv%-kEp8)36#8ujhwQ(|a-5@DgW3_4+Zd$rV71EluaWi#WL^#oNFctMVFjY@_J=tw%k36Q#6G=`zS->7Mf3qTs?*lw3jwTSN7cdOA3XzY@@;z zrl!j5N}{tZv?vB6Vedb}MKhMc7HLaT771R)NoBt@a$Uwr<}J5C`I3FuFtSk-N2&4z zbW(x)eGtKE9o3P8pF9k}$D?wXAJMs^1QR+Zm_rjW*4!^l4j&5x4oegP34oN|Pa}~F z6;%)mo61OTvs?DdlZ(=G@{OJnv>OD^xo5`L|9n^8qMe-FsV`Wm$zD>Z(Q)RstDqAn zrJ3XlzFI~kWBW-LcpQEPm3K7;IGeriuHt5T6_$CTW+NlGcM$+;@nibr5Hm6Cn z^3mK7ug?*|H3F3!WRI13`lh-{6?=#x=U3yaLtTnrnX~ev2s(YobEmee+T4P@!s@3~ z1B_X#Qy$qYPa^ACbNnYm6(`5T@zCba?lCO>GH~wp}<~YLVht3=k+_IwW zYzwmpf^011LT?TmRpxq=(w?-;;aBPF6U)jXsl8nGP;wX9)Qa_YD4kj{nuZNn{6{&<=@v8Sy{%)C6WN=RF)BCl@9?HX~;3Q25PL0PLWYr<`u2q(4$w>kXo2k3YV!28xj@LkHwRGlo*iM78 zU$6~vj3UQNL1RDR4*h9>o=u3vSCPv#^eVc*&kErC=mwQdJWQXbmEHTaM{YCFO9w+J zQwqF}owfXn{FvwdFJX3%R78O$wgdEPX`0H3?AqMQpEP~9Cj6%<2OKYj=d_}H zn<0Rx-3{%SjT$Bmt39?>r;wh;YrxqQjf!&f^78z9GL z4eMd{Y@WxBk>OQs@a;Kw!kWz|c|z}F-)D(Kt%t}v_fV4A%e17onq0DdX5+#7zQ9{J;G|*ksr1I)T(Yyt@c(()BP+b+$^~+ zkL%f%WTw}V%Uk!o1pcnG&=0hz7l_T);=vz0_EKkow6A_AMLu)dflM5iE!7*!Qe;`i zZsUl*3iLkmTGUSoe(WgtezgoH4LlCiU*C}&_+4GiwuCKGsJJSyIvp(y!(hy9|)5y2#{28e11uLJ>&CU zvL{7}IfJy7y|K3TL=|T+QUP7#4RTXB*t|OdYtt2H!`|%*xXiO*UuEYrz5b;g+uGq@ zcmC{9a#g;7w5vW4OSw|kyLA?I7JHcmv;MuVK%h}>2YD3%Q7;$|0c&~NH3TpK#g9Mu zAZY&Xq?jW;aZz|!6#Asr-w!nyK!2ta{%{_HbY#_NPcX(b+L^K32)0|7De&=ZiU+(F zWB$Y%ZZ|?|2cjVmkLLQ!OJF{s-hlR?NZe`TjTUo?^NGB<41KYQ-!b5gH)9o#5&P!& z#o2ih^oJQ(KtbHIfBqmiX+}Zm`lO;C-NUDu3aCwA z$h2iFka3K-Uz^#J`@BB7J%Q{wie&L69(B}EGjt$_V+6V=jb2JlFanm7qBxX0Wjlhe z@nJj6IxfGUc(MChgS%kXpQ)nkx8*q8!(MgAc8|ckhyK8<@9nk*fcs)NCOBzKM9hRh zXO!Sm3$ut3m1l0UPDiofyVuc-BBl*;YyP+-@(&+iOYlMT*C>4}T2Z=rP^rzhIs|F` z&GNu>3(x^=M|2BAL%Z5f{Qx0Aer09CN4l>`2kKd z>gQ&LO^E5`O<2A5b#{{K<9W_39$vyLvH4TAhwqILjb}uE!zU@x4-NRO736g~>n))T?V-TiaG&T3>qmN^k!~&)-fDa~3HEFp7TP*Ifs$%fC6dz6Sd5f0E~r z7Qnul1i!A5#S_OEf~5 zYb;6L&_N{;gB7R)xY8UHjvgkF&GC##p*k|gXpgw z5q^fmGHD}vLAz~0oufoRn^SlGu>b1UHuZ!1u@eVw(MqZRF7;dl6sj60MuEI0iX3mg z&l`)CY$z(|=0xnGCJS@r9Yd%O_%f*raGMhLLu_FR{8Ot>n7=-uopzK`gUS~jmAO)` zF~fdUqD53#fe8{9M`O*-dbgvvu+j|}R;HHy>sPJKMq?kq9*9#mbRyHLr!6NM?Y7dQ zU50$AJ()3Qg+#!&_il}a>sT@o3$@R6jPcF=3)^Zkr2JNLs+G#y$mt^Y;httSzl zCt0&%oJ9}NiNy^a12TYc?ZJ?mYtm*iFnmGu@@pldM(ed@PUsLKUdp0o#b)d##k5_X zh{FdT26~2LY}90i`Id9{s<(sA%{U&@^+k#j`4tQ=v&%RzpySFY2{m z>A(OH@Z#xt4W?%0oV`@T#nh|GErJ?rMObDTgKK$LS7;TdwmPAfHn-ug>@p^k;3^Ve zWtAo++Gh1&vDkWDYM8JNt8&H70$b|1hfao;-&9{?RUALKM>qQH=27v>7JwboxW8m8 zRQ78IJWI+p8lH1lP#(Vctk%QDXY`npwxj`C-BXEt=zh;T-s7_&Z{||xnpO%G@L(#D z-1-6Vtq8Q0VzvsWxo_-O=f5}(LFl@;la5W;QMdb6x%#S(0B?Km7n zM1(VmTT}Rnytu9RTn6k>ErOGH=s~Zsunnt27-B_*3P{2$>7A>u_L(NEnk)HrO z>+{37uJl(&BO&Nati~tD^G*&l3pT%Em1BkGN z+oo5};8An#v8A>WM;FXfS95PeZLDdlM3n545oFYyB+1}pj*FtJX3wxT*`*vcQ}P8+ zdLr)Mhjf#%ji#z|rgp~+H!W&`= zS*t~;OWa2GQcK!&EmiWNtE&g2GN$*Cl#OqykJj>gMMexVp;Hek?jG(fB~tV}W_D&y z)w#3yBh{5&D=c!Qr)OTNyX63bYBs@Ony(oDxirKsb^lBpEIa=T3NsdQ)n!U;PN?!oazv||pJ}Z@P z`Q+ZdLzOS)t;)N7pG20ATAu=2hr#jBGrKFW;b;yrlQiTN){U(5q(|FO;f+vXOMNM0 zeJZ>))I!Z|n8ME%Hd2g5J zm#!rqp6eoVKbw#Zmy3gk#$QXOj$#8Yu6HWh`mNaBCWF{|=}l*9wLq}ZAkrJQqNj(_ z4C~B_)6~Td8%vONs%<|9`<&rK*YY9K;M#xZoS>d6tnFrvXAKB1v4qEl#u`V%KgejH zgSMg&l`&&A95GMVCb(b&E+k+yO`8f|9iu6(6lqd*FTxd;Rl%o{NgJQHU?n*#hUMJE zjPfYA>eXITB|xuiKyGG$rV?VX@fST1`>P&|tNQEW0Wy9zs3Dfma4#)?kb6||N;*{g zhU6>0OL^f2%>mBJgN_8dk!8dR22QE3X2)qA#RnMld`{h6C4f}Iyi;j(O7B*?q^fyJ z?^FWScdx47eF2T0wl$?xVcKr6P%>-Ig?R_Y49nOYHXvhY1_yjpR3n5aGWBKUv4EOYRe!m%nUABO*vcOMyqJ({ zMTHzeA={Np(ylb5oX{gQI%eo{(!&^e+~8jA=_E^8`b_k&tc?P`V@pq(#C7BqUe30#ef5DIg&&-Q6ij3z7oA%fkEm<^7-Yzvt}P zIWwOp=D9O-H)bBBjIh{`&aIzyf4b#hC&=<m=6iy;$elSt0<6H6=QR#)c~ z8+7A*$gEd3H=)d7>-Q%Wg#@#ox;PB{CB;N8^x?ZEno8i&8#D_0wt4FURl5^LUDLU5 zn7?h#j4Aq{-kl9K8%P2)fbIa%04}9b!&1dlGo?PJ@w+2z%+S91Z$m`NfphHa09iyJ z9WUUi@aa{{D5gOaek6a@LU+KTeDttQ8`OITPq=f^SB$vSV@mC;#-TmO+OQ zs%JBj%yj+4o5gky_njv!#~(I_B-h#Gx$FLBb^sK&?%i!&=1NB8ATtRG`RaLMCfg9G zrZKRwB)b`(matm*Ql*(rP9fTrv`X$EmB))pbd0v7_hOxeQ%7qW31Nl)skWo~j+{Xe63pE?a?eHA&@8}M6O+K) zEI0fx_{lWRVajmYeDcu!@col}hk7^%4`AYeI_1W2tYM>nm6=g6^r+7DMr_-uH$iva6;}Fbgqc3 zZE9%9bkeYyeQ0-wXFN1Y<6C$o@%=AEF?yMUS&1F*nWBB1koDvgwdPp^3Nj>wbQWt> zu_q4h%9x%8?(uvExL)IU-r6MEb;e?H-Hq3IDEnqv65oQgD^ZpcE5hwlYor!ALwe0G zuH%ke?7<^R{s6K*Wjd+}A#IYBcBs<>(@zSeSxB$U`$CDCUt0`BuXaiu#C>l*h?9*p zsSnhp+<)s=7epDpf`%;H;AqwYQPK|1NJ{@>PYR0;ET2hw?S=8uahW@NB;ed$eD;o# zEi2B6PgNwr({~ztkDh!^UyzY*e}+QQDPDzFqS^11F#p6(rR@e{Km;I<< z-aop=po-pSTupND+t5a6TH=dj=~~|&cJ@y(n9+VyK4EMv=p~bg8l>-Jdns7UP0bNu?Rb~?Z9C{ z-a}p8XBU55y<$7Q`QxljABP1|gU32*m=AZ;@6OjyhCad@v6GWS72} zA#6wAHIV^JMCXBfLyu9M2hE(AlQI!4)FyosRCn<5>QJRSQui#;rxgyvp#~p&KK*`B zO(iYSY&eX#Po4JZ(N_yG3a@9knZ;)tF*q%xR%uT7yQ=YYallFYL;eKYlgPG%f2V0* z7K2O0NGfSW9sBD_g@&lOIhO~Y^ZS)@f+)Adj1qnovvpjAYSu90GQ4b41FK4_{h}Y} zIX%hvULwyRJpByRrO16-2v85VY)XD&6gLIwK|(TO>v{^G#|}jfSfcVU@0rEbr#7sz zy+-?{>)?k3=7i7w$i6R4hHq;a=nKi-FP~u%Kl>wWfeR@%RG61&A*@X)?+y-ooNrV0 zG~cE!NlPQU+w&vg?axuJu)cx$PYEW%%ko?`vA=L(0B*Rr)MCt?b6x2BS;)q$MVl9n zCyVi}JX!=y*0oU|*_+3Te*kdl2^VO9hMn=7w#ikftB8}xWDD$k)XpVKG5hu_25-uI zS~++p%O>0HMfPYZ-xsugJ5IlxAP1t66Q8kQUA1F3X+u6n9@|MbokKo ztdm{R$u@4=GUw$Ueu2+A#`?VAQDL(42hxN;6{N>l1;X3d4=B);p(H8yxip{5A0zIQ z*z-{$4v9Q^c{uo!#>$c`#o2}jdPc3sB}+#Cq|CUX_4(tjFWLDc7YGEXk&WHhL+{Q< zZcUURzj=(;=*NC)8Pw#$s=t>o8vGpcwZwX5QieCWp)q*Z0j-*dCIyEY_I;0g2^ZCx z=n414SEDm`!{XDTwJ9i!1ZRL&%9DMiP@?M#U86hhk5THnSC168C7VUOm;@mYa2jS{ zb7$PzCtSuRXm%nP)ODzHXV^t}vj>p)+V`zQtB4G;s2^nhri5miHEh^ z-nJ$B+uP^!VRTOIscBfM?v9U*+8}<8Tba7s;=K)us{|7wVF+Tq1NcccEUI**iuO!t zuDN`lXNsFxWNbU1cu;zM^yxMua1qg1OfLML7ym3!X|BwHOh1yB2*KdpQY6+AcE@k> zsm*Zhh9Tr83Blb9fR`H|N`p_7O!!U@{Y9-pz1P?xE@ecYsJ0_Y*M)-4k{O93d#b_s zN=JC^`B1@Y8bP@`8x$$Z@D(>dC6s?>sGUM9~Fj}w&`dOo7bP1Jp|F&E*A@mb^Yw4+)Q;VtC6gisUCEQ}3| z4{=bO9Ln!SsO0BpQ#Hruu{%nqo7g8;ua%WQ4~F9^ERfMaG$hxZ##f6 zwCwk~JnVk2J(pSURZVA1x8k~^(ZO1jd>o*Fo2X=0-OaV6m<)v~aciv3s`IN+r&wpi z)FL-u$X|qGgp-PxX%^5kydKEmZJ-Hlxj@%z<2Xo|TBvV$)Uz<5fwwch9B^vI_}!Bv z-y=*jX-{mw-qxuGRVBRVqJQ?u3w@k**__>`uB-sj09|IG$H8Kf?Ktv1wp#%S3DZ+5 zZPHsS$Zy92^`@m_7JF8J={M$4!rlZuMzA>Uv4jKr5OE>4oMk%>&yk+5Lb8rMz4;=@ z<+m=xZvK$=CD3p^1x#GU0u)K|K|St{l3a*=@ceZw@&QZQmj-PPS(8}qy^<`&q*nuC z&3hVjp`|9ma3|>p+_A&qPV_bt12JVvuLqqX$|9!)(bp8?ww$(W2Wk(G>c5t93(wr9vVIohmMQ?LO1npqf5R#pq@Q&TRQdWPj+ zG*u!SZ6B#%Sv!krRZztSAp$6Dv5DoO4=m@1_+i1qT~~<)MgkA{7JiZJg~6i`s>DdX z#J$Bmj5=zudM2F2hya6#kk7vsJ0tqF{v74_6w2vrMRkszRZ5<}ZGyeZIU=mfpA&xT zp@j!kX*t|Qtoozc0^2bm5Nl8&_@meYdCQ?%np;?2o4dGU2$h+gpRR>YQv|plVmTcg zpdnRr@Q$mjOHJONK@ie8MMMcQ{`9j9zKp92Txml33uuHPA7q(2R6{kxQ>5OdQ zQ==*P&DY2e_4L813Y3xUOheOX;mAZ$=Qu%$gy?DY00FNl{S$ua-(?{+xvq`kc3X%~ zem%YKA3zVcxWcq;Aazoa9=lgoiAiT6YI;|3#>Lf-rRmajtHo~!JK6^%WOgj>BZ9r zX=H=VQ-ZB&eLD(&zaNza4oEsuoHlWF{f@5i_D-q)^FXb`*q=k^7m_R4&n^0R0YZhb*mlgpkd z?TepudAEAn<*<68Et>iK-#xl;5>VGt${?O+8N7 zr};I}ioLR<;wbq>ldWT!gCjO29v|_k(Ffg6=1OT)8J0q~H7BWxnpZ9$VCueyo5K|8 zMB3TnVO2xh!KcQ-g0Gex_?R~7J&Te?a7&PON+4i3v!^^) z3d$Ok5-l)!bHKzIq8k^(Fl~*ablMy_N*`*YDwWE>|30)P%*VPZwi4+Jt1(bR^<@D4 z2RlQATd6fp@uxQqT=phDqe-u;E%Wko6IB-3`Smeu=`O3PG*Vz0c_k?lK8X)vWuf0og*OH2A7 zMXsHy>XeZ5+|M_BLF62HA?&~3xC=?8=W9Oujo^G-$>-ATCxgz?`PG%WzEpVF>Scp0 zSK&CA5#!AnG{0|sM)F6B$GESa7^6vH2AAHEsEYZptB#lPm}cL;r2g?dbK{!uXLEhe zhA7A49~M#DJ}5X*$DS?%Pwp|aC9w*#PdaJ!Lq>j&^+l3RiM=5Iz=Pm?FDOCMw9^dx%Zyh#LB$Hv`GfNw z6s0v{Meew4$K z-jWCz>WM{y6dTN_Hk^kv$qub8+*jgRKKaA8xy-VB zkNEWVP0Zg9yn&E5W}UD{0`EU5nB^U7j72dJIS|d`E!e;Q#i`0Tp^SEWKpBN`S z(9QB*H`i|W-tbeFvc86acAEoEI|86SZq|LnpzU+_KI0L%AGcE*V)aQj_fyczn4Zvm zn@{?6K5DOwCboBovntHp-Z2Ld_H}VlcHW4anN38}&F2Z8#2p zF=KwPB)O+($R!4|?U+~Ja=@*z!dq{-K)m^Rw$v6m72#?NE-UsJ0@mfO+(o}W14{6; z0y(f@z{1bU)X^)i#KByl#3`@XE;p>y)4s&Q(#_i5rBsgmKdDrrKuNDp7x$e($zTZZ z){qMNT^PHO3lq{;t0kVZ@aH4uvoUS$IRxd6_p(Yl!^}SdZp|EZjJT5t$Tygh>4q7y zj$4IDs&9yn$Qt6dS3j!87g#Q3-?vUMGn@r9=Wdmup!SakpTy31$WD%&j`LErce(jr z=&2nEC{eO*{hE-@sYHYRuS1BJJB;5Lu_9f%R=S%ui*QvsOWwII-E$v6zN$+L$w9!q zRT+R>2n&P#i)Jn3{gMVotq|Z(Wu8g8HsY{w2ECrJ>Anq^s5+uOA>{U+VHp zFC#L?gFsqY08V#m;7%DzsSoTm9^8%?k;O$DDg+V$W~&v2+ac&6FWs^3N4-+QQrUa0 zSfeHa@v1H~58=9WfP5$ng`Hf*=PxlX+#% zs{;if>m@9mD@?hDv0g@8n{08Syf&0&6o7zt*%zFPCM|q0CSZ3UR1iS9pN9V5k+J;0 zgK**?%Q& z|04jol=yZn0nc}5_Wu%5H&)lMx{h!5eTR@wH|wAf=TauK7kfiW_hjDF{=7c`2^Izo`cx5Uv{_ zI|W?pb`vvP^XyM>iGb(ug7atD)_fXVdn1&e+i?lGnK?@(#7pY+D zF!-YThXUWzymdI>e)crng)dzIc{4FmnztAyTwJM{7%qO}B*6{w0d5ea01(^o;2?Z+ z3^*({m?04A8z8e7T;o znzyYU92{%B>S7m`zH~SU9)r{JCP-ii2bHl&fr6&15%KT3463^sk6mM6*We!TybY83 za<2Yeh9MBp)qk%vZ&ee(zl{ODngB&@#7LXsrzSvG8wFfUrvJr81n5jW@_$Dhiz$#? z$P6&0W8V4SnR00hQ0u;oJ(f8jZb}Dy=%$3LqIGM39s#O}0Z(!;{A>U6*4?rKL2^LF z9TTW-y>gewTB|}Hw2K4^NZ)`*U%|D0@8Z64S@HX&Ek6gamgUBrz4RpBTDLb7oC6vE z&m6FKgu9>DNpYoyrt4CTturtaBm#(h$GoZ)?vF#5aR{5&;r8U zcyJXro(MNoNC7~xwb-RN90VZ3F&(t9Z>zQiA+XtX6`C85xf^QuD?#{g3|Hx}e#w9_Q~+&q-vF7vg7(OP z!gjjrVbO7EtG)(q>yOi&OIueho8SaW^PaB<2LTDtYef3*h`IT!{Hs09+qw~aaR delta 23646 zcmZ5{LzE@Z5@gx7ZQHhO+qPfXwr$($vTdWwc2yU8`u|zX`Ll|}&1Ia-$c(tl6`)}y zpa@E`pkOdSKu}OXuncOFNeJZd|3TL=3Nk?;ARx6QAr*`Z?R%F87hs_OSNcW&ue6#7 z?7uii`G4_)@xPd2&2WJHpNG-Cur)LgARs3&ARw_6CRmIV#c^mPRc}`}b9*HhM<;U^ zH*0g(l$%j3K$)t&BZ@Fm00vdW_~KEOQC-^BnvdE^wBw=?4s=`;G_!oQyquuDF{ov| zrfP|%HQ&<=A8*gw@!9$v(3F=ANn9{AEI6nw^$tf< zd>k5ws@B>qKB{h~EeXQ-V0(}++$5&KL6v@Zrz)QrKx$>E+(@h1H^^(5pBv+mUhtLL zWPKRo4YBL_IRH(rvK^DTmhhRBT;bJx16jqJgygQ(@TR&QDs}yUCKYb&iU_Is^oa^( z#9JHDeA-@m?>THcrB_tVlkplnU?L~y304CuU038rwab7i$21cY0uEQ9=_d&A+kq&R zm%qvca3BCG)z7Lb+f@vS%DZb!H6v}tgg8`h7pa*~pm+*GV@-|0qE(9HYQ{|0o_3ka z^Jn6YgczQ$Q7Kr~^B%JEz{YG`Xmd!EZwa3O*K4Tp6EJK(@vXrPTsxNwB#CiWoDvzT%2&<&yP}n= z`r^tU%UP~=(a=H|;5_nNw?wK_oF|)ZfQHdNkMt?00H>{!5<)a?u2W|WVy8!%BZTmQ z+$EzNcQQcyv!{@2`b%MJj9bOf=>qPwBFe?2Rxdy+eh|(fc8Z@87uVW&x$Sp2G%5WS zh3>CeV0-sO0IcTZ*6%Xq5%kPX1&qxV#{Xqp3OCH@e<{}Dr(Xi{Uyk7&>2HsM0|EW{ zM`70gatwcz2#}^B>x62K_t!p~!n)g@Scq834k)ZuDVPdlIXHd8#7d_Sz6>0_W0FI^ zdp9meUe=#T;PTV`{yeXQg_K?katvEiay*3RIOt($6q_+0i;-YtzauBTA0Od^;P2Vp z-#;9JKcB1cK#0dFMD0Z_AjqX7ODB?^NCtRs$6`nVZh+`%Y3EDmLd|PyVQ%vDejo_TjN)tfUPMRzx{Px4TevovP%a#{(7HCx^6B-nOyyiocRrilDvqj~@N=6@xCLRl5`jDg?jJHQREzXzcIsKBo|ux5 z>NE`qclCEx*66(ot?bT>NCRK3k~FQUR+pySGBbB}4hQOwUYpS^Mzd!VVO&*SHhueP zvlWpdS?1JS8{|qwXDF1dVh6lzY*ZKe+d3;-9{`;xPFAsbRXDyZRW}1)JKIdmpZ5cA zcGKm|0%Bc{oLQ^{O_b!%Erj{CSJZS1h~9nI z@WORv+jA1Q+C| z83#>c=6WMF0eZ;~I0hL>gQ@|&#K-b-&4#xhx70W$erL_ZMJzY*t|NHSPDwdYLJ;2d->AbGr&y{dR-bpr?=4Z@HO&jb#Gw}&DB*L0cz5!cuI|V zb@;(H1Z=rM>ndtA{|vT1!@7S-cae`l&|!`o@vu+d<8b&X7r#2Q1EKg4f7 zgfA`0chD)MF^_AYeX%`0C=xP?yt70s4@nV|D^4f--}Zql<~z1H6~ycgT>uzG(_Qsc zedkn9UK@k@?M??taCpt-djb`bdYtS=Oeck*xZk;oJ{b72|?6qp}~6PO5E^WJyAQG3ASY#CA6oOHn z$4g|mB%a{$L`l~Dk&w<%4Y?yE{&P%L#sm!?~cjOkY2&b z&zN4`-`}G7l`e=Nt$rc-uPwol%%X+~$zw8e8u=BV&ve`s7E@l4?;He_b+If1#PJwo zy@&%S%fzY2OliNlKMW|T{*qA}s9Z*AJntg6(+@zEVen7y9=D!*R@(H&Lz3x*bGnC_ zF@o1rp0?Z2$AMBx-;nnuY=6QWLy+N;;V?}Y`t)om##%%_kN9H=neHG_fj(QQkLD>GBJ@#s|?eHs^!}X z!&sx2M^D3mh?bjoOQN$$c2}Ui$8WcY{0e7YMnMg`j&+*DVa0aY8oMq{%3{G&&7YPl z$l-W`@fo+HM3?0rvy&J(1>lorpeoQcWqOLgwwgX2WRn1v8mT;vU^ps5b!pWbSy?e(pCbOnV zE>!l+`U{O@!$Oobxx#H_w_mzW+{{pGpAdcBy`Z&su?!YZhw9$flJ z$iB#`;`a!jbJp#+0+uE!yfdO{d032=?W{vHg5xb)lb@|F8k3ft)&K166jq?Q)q@3; z8+BP|s6p}CwD4#!v9>f%dLZ(~--Mnm55ZX4nWi`2%eW7_#-2nXKzd8-;e+d?0MF8H zWEzGangB`#x9UuU$4LYhobd^(-=)UKJByA`@e6{r{le$n0eFL$H2o{isxJLQ>oom~ z5A^-Z5Bvd2&a&eP@6y6+YBwcEU7*c_F{OF%7a{a-q&6~IfF0! zd9M(^N@?C4z|*}pd0BV@=OA}FQhjKvN*o=GZVi@;L%Jmrp?{jssGUvC8B2d_QUMaI z+hI#@8BY93k!K^zpJVG`>Bt&zV{-8jJPiqJe1>C1$VSJxRQJ~kyq-bs4FYU?t+2Q@ zzN2JhjVXCwcutw@qi|mN*@2j&A@bFG!)}Pcc}h;=-6Ve~WqP3^V^P}H)iwuOQxXFQ`r_IfQcgju zy?5v_^Nwoyn&W5gJwT6~b#MNIywFu9~O~ zGVg71A7$yQOkJLuU#t`B-2UgA^1MGAYLfrCCCIneqOK@csBo(iiEyR}?*&46Cn_$| z7{z5oCH>pe@)ZR?L7{nIKZF(kAWEq-4C8W)Vz7c@gA@D_=N^G& zlE^!{DA&-D-qG8da<2@aAq`e*$tSPJ)F3Ir)pKYY{x9VJHpVhqc9M31uAJCWf>sl{D)8s#sma9~yXBC9fZquq{hRu_?KI1;kbRtY)_XIYtou0~e}Cq` zIe~M>29CI*hX`^CJ8KiN~q@%ucie%$f5d;ed~l;MOU_oCe1l+isf z2(el*I?fXSC%tVgs;+Qz8YS2As_hW*S!XxNUrDkl-=d_$PpqoLDNh;Wk`ZSW<&>_b z8E9k@4py9PcFL@|gC z=C;~weyrhX{zFukY{=d_Oz>~T!DDJ~wZRL8X%$<5ALQYcahYk@ZvHxYH1`L_155Vp zME6E)mkhY#TfC-gku}e>K66`I+U~r(g`Khhi8eX;oOpy%k8TXdPuT7rR>G$$pZ5F* zO9|gys#H69w|UQjx^+}X>CWGF&OPYh?z1cIl|=1@+agG6?FA%We<*rn`@=!2Ktv&f{wehc#WE zN0aS4Z8sNh0IGt$ji)fYxb+BBN?6B5_8JkIm@jEWg2>~pzlOH{G`p)38o0YJKP=&~ z^j%ZUgeRNM1y6l)U%@5jbm!z{&D@|gm%5Gt7#xzkRCC5OwLHL`aqx}8dapfi^W$hf zMOBmta+_sdiy3mv{vaT{OhTh$Iw_BO^AU_DsKFhN)8r4cqq}~J$+AR8}+WW7TpM~l)rzzzp6d6)GrN;HCJEe+w z0!{kH5>z|G`)WJm;n4%K61Oour&|z!sw4PbX&11(1k9EZi}ei7Ybt`u_XxCekRNFW z7}hy*>AuoKGTBqua`zacWXB8>@QS@ddGc;GPkMI5wboAH1Sk5cZnOsV{Cg{WWK9bL z;XB^$1K@#D{jN1NC*mHPZby94mLG7F;YVf&SVB?2Zhvpf`1(_0-*(NmYD)+pIfsgL zojxa2^*)Fv$pF9V#OYh$Vd1Z}MZoc-cfu)&DPNzjgy&5-^94(QC|p^08M%ZEt@mtM zm(%5Cpcu|1??tm**S`93R{+)T{tsLX#8ojq|9~j32YtX^dpUZc5W^G8`r|xlT zdb`)d;VM1m@@vYw-%e*q%jj!*%U)_VML(01BmD44Tur2QW*_@z);Dlw*%X z>(RFtTrG_z{DhOg!*+Lupohj3^KH|j%NLPmf|CYU#4Is#o3StA%=gvtJcGJ;9da>5-9l5O=Q?B$ z=7xj8IYIB2>_ig=JfAxNRZ+#}w|0u+mZ{t<(TikeFw@QTC{-C0`^OIWWalq{NNz$| zRAW=DC^{a2EFSbgP>E|?R#lhM-3ote{d(#xo2h%sl&q$2L)oZtreG`D+0ID-F*AHV2qT1u0jz{1Q(%~+z8rQBjH zh3LmdeXG5<-I!p0SrRLv&|6R0+ka$0Eo`4F=x>I%0t-v3G?MT<{>V~8kuiVD`HRM^ z$eY+czZtF9U3V(^f!`6^MILf}!RGG|Qbe0X0oif^$%b`Stq|{AqWH@&{4EilP(LN$ zU0wkG-ILaML|U64+SK3w@3hIa*S?FJ!Y4=%fb~N=X%nRE+1#WXrbQZ>vK1roj^M;W zf|pL*KuWx4BDys5Opd)Ky`4)ItJ8^*S?yQSUDT*eFrimhmR_LRu<>tyEoy6PTh*|2 zyt?V@dwbp6{-k*WSa@*em?0f84t!gj`sSYf&Asz8H2D5pmW;9@1|Ss*%w>#G#*xGU z46}nl*nq9aCM$3XSqK{L!4MyE=E123QO)g=*-}9DCR9taa1KcFF$#G*wX!c@AVJyE zEGNFQVCEm6u7T!_0WvW_an6ppsx@R_Ozo4OlN7t5ub!kc2z`5g>B|E75uwLcEHK4ka zhk!k@`Pn5ka4%R#^>>o6ZL_6zpt`dgV4K4j^gZN4jl=DsjQ0#GR!PiQ@82f)N=2*B)xj$P3f5T}PBW{Dt%3Qk|~8`(!Dy zeFD@;mjf)v@3~*a%dvY(WNB=k*tRo~%9t?NlfQZR%eR|XJC&{Y4nYzDPjzgjd4UlIVSgQt#X<)-PF79d6|yOzvbx*#a`vtFJT*nVZ&H z?oIa?q{bf4CtEhAdg_@2V3t6#t+GkbHf!%bYys)jJ)eaXe zmQvSETN??$K@wiN92!%`q%2*2eFOS`H5+h_!=+H!@E zzIi*A8~x~mDb=EsEiPJ%0Ym#X%Q8IW{yDwIg)I^jx-iv*I#n0M(uz@#yC&@_8@`58 zq|T0^vmK7Xm^O-;KXlY|y4t%pDyo+boq=xZ;pxz7CPh!!(%#&;s&IyIwEEm@lU2U- z^c)P387T5p=aKuS`OXa}S|Z{vS#p2IPOC4nRrd)cP`{AqCx{(abDAoiuWrq8P`dE> zQ#;h7-XG+s_Cc{%a)0*w*dopGP*?qjZ$J9xV?<^av{w4uU-*RaF{4ntJ{jtKt<7D% zQWfo|dCK>L1I*(#G}j3n*+=+rTfIMIp!gA>`Js^Z_Mkt(aBChiUZ{Hj6cs(1^vw2RgFXz^9aEI^Wj<%T$VK2`ySE1LDB zA3U?&jpD-4@yDUtrR>FPR4@Cbkw*ZC*vek1%gzdQMkyx6Lro{o&JVP@2!)8llgUqvBLP z*`;NIq98evh0g)u`m^p;^%aF%}gOeDU>cPw%?j&zY@b^0UPtug&X+hOVwvEgB+Orpll_Mmnt^#wl8$^YU7?Shi=;%!*$%5~$dg!0`?r&K zxRd78sxo=`$6P?PCKwf-^Aw%}K*f4sCx)F;3(V zs_AA}RHLuRge}l}b2<#LThRH4?a-#GG1o`-ba8Pkmv^-}BEj)Gqy&R4XtBB?$`!R> zypA__3IlHGH=_?RH>0$_#)uL81FJuGYEIb&mvxlfpBR4+1xwM={=tAmN<+UCv@{0FCN% z(5yT*(Y4UeP4dWCgMMJT0+X$(!b4-Att3APOc3XPxKzNW;{U z3s!jXp(7Bb#&k~2*0^+mHae3n6fo9hSx*%8ButTXpBgL8!Du)5G;@`?pWJBzQ}l94 zBlXlXGLJA(fI~kst2xw+UEZC4D_%`4DteYrdj3O>mvaNSDoisKIjxi>pUl|g#hoO; zT`Hql&Z;m=M#s+O1}3Z0WvJrlu=snH#M(GC2R!>gK&F6(Bg*G;*7>NUB#hN}#jAj% zApiQJwd6c-MX%h*whlJzXdbNxiKL?M@{a%X)m!xS+gVgoTRiM+zG-V~TB`u%jUQ+Uc~Z zOdaC)aY(;#_Lc_0Gx!rNZ8bz?WkET=YhD*JBr}>GV zxM3-PQMlNTC#PS|SD0K#_MBs5J*4f#jdvG%)qEd5z4e4<8;SNaDa=1X{g@zd&;84tlT`Uj?=Yp6m4Jru_C zvff=E8$+d>&Vr()9X!nvyW^FEAR*U}kJ3d&GI2|7(WF&}$aRbrNgFWE_QB8%TLeS- zM=NM}cjz1m{wdVr2U>ZY*@F3sFopXPy4B}`C%w2Gs)x|8>Bc_>SXF@xg=Nq1-3ecS zVz;$g&V_F#6cUIUfkUt*&&0#pA*^ zut{^&3g3J&#d?K^D#5ztmc~FQ`&yr*%V}E3%JMZ`@=8M|OnL|+3(v4!x)^_j3 zrp`9{f$Fwd%;#vDwvU8mlwZp#cgQ*%Xf3Yx6?*XD>kC|5CNw0>ky?DP#RU-nEIXh_ z$9B`f2!S*Rq5uJl{J6^ADUBB98Vgc>^D{he)UUHmN5yUDXI4W{L_yyNbqk-8tSP7m1~&KFU!&Xr5X_aii=ZxTiv{bQUM(< z6Pzvf!5zYN+lM*4m8t!~jXyL1u>O#_Fs&>gM)0k>ju1B*^}%k*rT`wNfEkHhxc6u7 z-jf(7ZSbq5faj2GFGj{I{)#i{XTNoho40%4d;_N?>H$y&@bkY&kvx`!-iYuj&bs1< z)xbQS_#JRd-uC^LRrRx~ojCQhwGqb$1L14HHg1pGIKMeNZ@+rZ;mEiE=80=VTJm9- z&&w4fi(b#Fu7Y*ELbec9W%h|{9g|=1pl*ji_Nj7<(cn2Y=*;`4?K*#B)bP|`I=GY& z|LA^NUupbCmxjV@!z;8#*tYk=KkGHv0oh19VcBO+?t;TG2+j~)=!1IrLUM`ZD-c3( z9`t}lFWE)B+J}wVjgUVB1o)1bi22hW1+cHRk`DPIovjo1I||F)NsjPEQh7nW24J-k z41c0FJ{y0*gT#*mjd%j}?jOtgZNodg65$LeG7ep?8-IF7890t@4mIVj3 z7E4tGiWkZXOoT3QL(m{cdbY#k;1|$AQ-y$$o@F#y&T)wVvIqG*LdWf_CwYPsTBSK$pC~LzDce8LwT+Ug)d0ekN|TOMcx}W`z8BB?Yxb%KL|DCqj$4g z&W!$vdt9%8;OapKl*ke4OHeos3$x7M3(^bAfRGxWZA+sVBHZE zs@ML$j@c$pJ47QDxxZ#-%e9dyU(fS}w~-r>+ys1pv_}{_tDhp35lhruW7d^i>Sspl zaF}n~EeaU1)LFKkj=*=ri~VI{m5_>=sf>}|3Oeo-T%}F#f`wP-qy4J7Uz}JI+Il$( zAObyKPKGyNU5qtp^NTQUKRG|+aq({lSQlT6--cT=xJLNW>;akBFO6r3wB!|550=y) z!jqdiv)_#^EjgbTcf+2p_uO;8FX#9?ISB9OdR%s(Ynfd0bC@^sUZG=!z3KR#Hez^$ z9vujFe~DMZI@LCxE&c`lUw>+MP%diZpIx5@4Fp90->}y67|uV(UR!rt1BrjA8N3>0 z(2g0*2?Q-?R6$h+w~!4wtPv~OD4nM?rjb!XEWMHr^*o)y`V;rhRz^PuV|as~{Qf87 zpB2EON4;4H=i@4(XI2)k>)cGA>G#LHBO`Ec5TW_Iv!(>21z9hnv4#ZgsoBnOZ|pw< z!B{#Rk{Nqi>ku&f?ZbDF9Py%+hLpiXR)TDcvSw~)q&fuoBIU%H@ZLBYG5+$dG$#43 zHAMNM^x>i_ftie=0^+BdU9`Cj!dG-34&*#0#|#|DJZnVTHN(-ewql=0_LG^V_5#1U z^xjW|cB4;EON7T-=f-AVMV%*JpmX^n&{FIvzdX0qstwR!Cv~VvpRJV!-lH>wIY+b~ zeO&=%udb2c!Rer%l~U|RnT?&Z=E-rpuhhsUXsd2z}Q~Y zUW)JLX$IIaD()->FniafIaGv~cb#nxlWljfFHguElx-}=s>#h_PwHE6oB4>Kdsmw= z+*&EPt4&{0Lh7pfpqXhVzlSQ+q{=0?VHreQx`zK|@Qz2n?Tzn}xA0gsr#P#~;Ch8y?Yu&)>5@xC|#?3@1p0 z6QFv_4spWtk9w+IKLOQn53jLY*BXIXSkC0;!{C(TuT))gz;5->ovT1$_(Ndc)k3hB zPyrOIQ|;|}$N9k^A)7;x!RH_NBVI!SfBpSvJ&(%;r4|0&Hv6JZdBHyWZwy3$3 zdsmM;o)YJ=*~)e{5-8&Jt6_`v-Jc#p6Fam?3ha;OPzfnbf$GlJo5wY#(p!}!Bk3!C zgwV_XX`$Fvqb^~^UM-IhBG|SmdqqaxVWwxPCek8JV28>VufOmET|fW3hWPU&bO4}_ z8@p@tf#FZQ}c zuZ=Vg++f{!yU``|=Dxpeg|o_|622fnSJ72i(}0Ic$gX%7c< z$oztDO`pG8cd3wjg|D2a_$zqijYGgZgD1S4vo8PnBRuPJzxT=OA_M~K zkNI_?@bV{DL$_v;*7sEyXKolJ~Xm^b#bbvcG5IQ0r;d@ykp?5My%HTa8>kF$L8-a80 z(+~Taagh!Co;-h=c?~rB5d|Vny@@tI8fW>F1!8~fF%Ce^z22&ot{eTRi4q|})r~?@x@6&+#X{HrF9@>byLvOAgzr*1a)sJ%C)1^Mz3HPCk zG568Q_hFAd--9w=hlv702_YX)zt{o`s4MrEaPhHkZ*cH0@bNbId3HK{t-PEnExpX- zge&h@>>JjSR#Z>{_{!6Ar4$_0T1)n&OX=y&9#)gF^+x9Tf%S7%36df|r%GoN9P_8q zy|d@-bGx{6e72r$t%koI_4`D7N=QS7`NL57dBKl@m&=*!ax_E~X|+x>^P5m6PrLSZZXth_Y zGg0x-sxg!=@&4r!6fLF|PdDw}|GY5rSTw}dYi2ndw=9B|b~3s6aBrbsqq83xXqw)9 z{QRFUTH!;LL+kb<7h3LyBxQ(ZaK9t(lBxY0C2ZIghQ#h&52iLLvm9|d$B`o%onL|FG7=^Nb zSH5Fjc8L1mKzZx*vK;OTJF@mUMpa>PKVdEPrG+DaXug=e4qA4wE>V&R^5vGA0_yy8 zd1lGnBZXkVuS&)9bWFtHUunH`Y70wWA}VVQ+li<>vN??W-Q;K~E41_^S#;g&=2#dv z%jKW!4^ph)8)9QK_!cxw^Hbo&A#2WM`i@T`|e6HIrpK*p43#^ zSCO@**x2Q(1yU809S3$yHtVsff7`%c))^krYk&Tx$zb+U*B5pL?h|k8HTxHd)?|(_ zR&{I1PwX|ANmahH2MqJ*J9%-XG^42c)Hk97TDv*IFhe8W7BZcdv)xp=91`WQHMI8U zm#i&k=_YXSGH%bu9o6~b_M?XOTWm13`Dr|k&1uFOWk<`rnxZof%4iS{WYDRxXnq0( zRkU|h8}xN_G#@DWN{-w+rAG_I^$ErkE##};W1PO!Ij(wh7Nz4dtk_t~jvV7-bJY0& zw04(iQ5RGtMZ7pa3_X7*ww=KKs&f30Q%WDsTyDi1~`t#xEtlJtL+sME;b5<`$f}OGSYB}?piB?-3 ztt>QaR~aZ%HAn`R|B>NtVYc;Vk>$3UJg&XN>| z*;7$SpxW!dir&RiGt6V?t<2y6oC(T@)|4OcWE`1`+A&mDQ|QNbDTStfEcVV_MuY6> z@g8dY$7bvQeVYKk5}|OxFT;G}USI8lxt>UF_i6u{iNIQfRv)=7MPm6hRP6h>7(mqvs8b6W=cYzdc6#`K$v&660?-%~VTOgI-G^cvzs7 zO7J;{SmNb-o-FnBIaiei3`~eNh)X$m%!#^mZ5!_0y=JTn45(KIhCAm+CgBKiD+1G= z6~@jMnbD!p+yL zNt`e>W{OPc(4P7KZHr4%&%%{0T929MacJtAVJh);XiE}UJbX5QZM}7~)5@^kF1)rW zk9?}*DLXR^bu@kXMM}J}C8;apP!jqGWmIe4*ewhkcS;9aYkws7B*;$w7;|m9j*24m z$I0RV4B8g1MQvNBHH}SMN^V<9URz@m4;hS_yiwJ>QHdPqxR&-cZQJJcY{VktlKF7b zqrpWIrl;Gja2IJ1}3!!cNRdCtQXqfv^aQDo1JyM`bjB3LH(7C5R*S znIE=GYUP->!YdB6mMx*OoiE6X$0=W6Hr-a&(l5BFqt+}ydYDnToxN_=H%Voy8#}`T zSnFb2aqI_w9IaKD^whXx_NRuzn};tv(uQc!i?$tCaG5Jto_0?FhULuB47QHDhA{%X z@`*?Jd1Rf6yI3bpUo;E1_&m$z$!~a>eY`mfkUUiAJXv^cJ2?9zBxrk&swcb4FTP8V zjP68Hj65{JqB!#IcU=*^xxy=I^?4>^t3bB+yn;$wowl7n6lR*y8L`^y_O0DEb$(l< zYHLGix`hjMQ+gN#MgTPE>M+tvOXo42V~T0`NneJ6p1kAeKGgS>W%2c>z~h~{yVO1S zE3O`#%)QNZN6rKYj_`-%LGgo(3GtyWd&C|yh@Ue60DIFHI?yi)g)q?(=dCMJQmrI8 z#xvH&fEk(wrrIqGzN(>Y9J#4B?g)B_0XodFQsW}tH#H}CLEh%YUEi$lIn$jj*nhM0 z(;Q?7Z9Z64_cNrB_7KW@&|STdWxMl)fBV70ZRqt*Gk@5;b7%mW|VlrIqKO zlGi5g%_7(tih1Cmn7P>*7!ENtol76frH`wpXYoZRiyJ!>ORKD~WS~aZQ6r+9*VcNQD+b_Px0n7xxz$J2%Ql z=_VB$QOibpy+BQBuH*5VZ?CwWK8?{Urw+om_Xi(v zM0_(ypBdM4I+b2goMibV`M-V zKm`v1OPaW2Mzc5bn_y-U0+>YZ_*;2^OP4X3^Vf!pxm~h zEWS@*1SNB@=C?j~vKNl4gbPt6^!AlBw6)Zf@DEoEiEr-jZ!Ts8G30>HlG5zp#7WlK zgTP*0LW23(pE8xDxr-leMq#T+vlfp-{BwS9^$X{5;wZq^JOEiyN%vAB zgz?yiDbW6YJ5r&ZTie2x>vfXHY*UrwN`X| zpcx)Ky??5%8C+vMFvez*kgKl64V2HD5v4UxMR*jjlTyu52F73kP8{iOzT;Jq%=QXl z0(`kwph0QwEhWspl%O>{CU|k0zYEvT#$1wWQ)nCjh2=&VL$$s7_Vt+BZ^UPHo8x1j3MSkkO%kyv%oM_7L{2ajSM z@*W;iPXeW@@ukI0haqDlk?}iNtT;aH=s8#Px#S;b_5~P=yIe$W+Gd?@r;~eOx9Pxb zIw96=uzu0wIE=~T!~+f~G)iFg$>0Y@mfh6rd9m4l(s)bdsZBWISg6^)U z?;>6G3J<(!Y!)nW|0d8@9=^SYgbN-i!u-@8GQQKM*qn;oETxp}{^<-KIK0F8_r29d z>Ieb8;=v9c<5umS5&rKO04&|7vq5$3L@O3^?znN64?7|24->#P(A7m{_{llv`G~b; zmqWfuu|q;PMcOnLot^mMXF7~Qs1EW`z1CVf1{b@)K(-?eU4+i4+PngC)}1QHa;n&@ zM+sR#9xYNto+`HTUX@)rbX!ZJ-nbK$9b|NqOay&2S;4d`Q!!3cEQ&%#Jd0$Cy-5lT zO}Qvarb!D~mJOgxUVYKZeM|ezpTY>K;!Mk9VTMqKBP>q0IwG!OxUZaD6=iPHHQ3I< zPrzBX&?xK~%Zx0Nl$kFVzuZz(VnOn|8rS3u3SU0cvMffXmmKiBIJCS5OkHX;g1T&}rqs4r zqIPjaN13)Uif=~;HF|rLA{CF5WU)lqlnX8HpDENra>=HgV5^&b%)&^pno<{)w33P3!BDCb-jE9&FCnP{ZBWbZ zZO*xmwfP8hU1i$8<`oaFZRW9PN{ho>dtI-^zuWW(2|k)<3Ra&jSu$}nG}T!qV96L! znYv0-V^GL|nK4%1e@`WB?u`lQ%G;w_=i#e>)DF0)8%aQ%_FnGEIZ>o*xtYE2`DnT) z5XU@0pbpjNq%>{NVBSUGQ?yhM&fxORhbN$O z?YU#1P?AFsN1vI2RSf^|Hn(ieKd^UWDWYo0pkHZxy>u~KK^gOs%P5f{nGBhAe}XAl zX98&Q3aVrqa)8>W+>A6Ot!djK)xzGCKV@t4`$MM0>XX>yA)Fm4T}O8@;|Wf5WUwYb zY^IO9pQe6Q&wGx+NGj-ORdKR%^3r$MPCu6(+-g_SrPOgDswzyZZq8le%4bsM`EoJk zNLQe??@d%14@kuxQ}asz=;tUrajw$;gaE{hh_RNj$7%V3ohL%ZXcJ@jt!{YDC==N) zL$8GPDh=+$J@X%S&thFx-Sf{>bROh{1!g1eKfLEP`iDXYNs*(nn$6i}@=97jleV88 zZFob5^Er3_^6idg+P$cKi}bSMyWD&+eGrul@BCf5$8qwU^V#*tI)$%t9xJ*v`UU`L z7=yM0-xEjv%TtT>XZL_p9R*O?Hn;$oYz>FLi)#I;sYCtNociJunSOV|qp`^e@fnri zo$6SDY99F1XNLgcOTO!dLWM_(n>dFbMML}gp}=y*mX?}&MQjwZ_%L+FNEiu6S!C8W8MN=p)7Kf$@ z#ZV|prixkCN|sPhq+F%{DIBmy5$@9dKaG5KSXE2Z_n|`?=>|bSknZp*2uMk{NFBPn zk0783NNhnmC6ttIDbJxfl!SzYphz491--<#IdHun_kI6(_w(%M%$fPE->g})_RQWh zd#_oY+M7nA(J>|yxwC?iP=(HX9=>&YBcbSIN_-8=NiF29ut~~Be?qa! zN8jkB1NmLb3%fV8BXRkZIc~m}-n=976E^N*Xf4-PLyntQTt(1eCE}`l#XP)R^6`yb zS`(L@I2mU41#Ow&`0r9us`1oIY>K})6~42wq!bvbNFA);_B^d)K?p@jwuQ>IxhAQx zNtvnHV-iB^Jh+)6bH>DXM^o4vL?*srYYMX$J4<2CAFY)R6MILrQllDD+B5HLX~7h7 zT`_+ExsOZDw5+M^;cSI^ME1%!(qEF8>b2Dv4H^?wJ5Ho+84CMps&Al#C2F4aRgC`f z6TkjWy5@W7^v^53%WFh21~irZ29j|;M^1#IcwG)=!eKv}Hk)gv^`Xp-RF8-DR=kGM>$Ljf|0WebAR>Qu8`Dg11?V0!X z%w*%;-v#rxG|LZ>QKS4GHRH~X*)NWxuvMKyq!~GV1%3Q{#dLrNabd%vzJ8Uk4EQvM z(1)$h+J@sZ)B#VL!`r=rHf!2izsRlkhH%y+JIDH5ehO4&^(_wV%>S&k9>M)hdBmH) z#>t1Ful_1o{XKn=7e@#~(aTDUi3ULzKM~6VLF2+W;ulC2HMioeYSp@B9DlZ@8w4Tb zfDu_uDzpDvrh;qR?pq6X#W#R>BGW5-NW!*d&x_L0hv!Ccdz^R^lOCsKNa9L-=lf2cD;#=W=H@jwr$^CP2A&^=U z2!tJdyDQI54s-Hjn53GYqBnjfdyf$VL;DG~AnT23RU(3Gns+E=9^DpV(_DsYn8L?c z<3+Nyl{v2Gk><8`aUd}ZQ~g-T?9~`d$|PSA$NQF7OonbMhvL9;p)7GW1J%o1KTPBg z7DU#wYj&jmmfs&a4fbU2P&x{sYxXee@ly;o>^V@BWrcM;&BL=Mx3PZ8P7_9_Kx8fK zuNC-<8(vtaD8jb<6yLPRhN!?j-yN!z(0NdR>S?BZ9G_%gH0yp9(etU1h{ds+6xA($ zgnx^A*Zpez)8C`Cq2UNCf3xM-*GJcVm}TV{6Vu&n&ELIf*`@hM>+7oZ#hx09PblrLb9Qgl)mBi9n!s zjJ6+>9`g4*V@PX=q9C;m$Po=6lQondl%nGOcw@7xu^aF?qwTw+t; z7q-K3W(`Vqe0o9ErElAXVAX<#$6A`t=B#{p&6sJ0|Mm?;N;DBpQSROAmaz&9`{juc z^?DD(I4H4hQPR8}N0GKeExh)^0T599V7|`P9ZpC+;u8`%dBxDbRm=2qa}>`<>N^W; z+`cRW={S8(Lg6&{_4O61p*sv2f;AKlGgZlcMDJEdzYV{~_3;2V4Epxe5%&*7bTMBU zPg^#uLWeAN1I6CSS-~l`eXHwDZ7dH`PZ|ecUZ0AkL!)SXoI4Ge7^<|DE1#OWmp<2+ zE?W_wt0*897&Q#xW)iS&=t_=@n9;A^%>4TGP|AFC_eyJj;EzUSZ}ovj(#o&?Om1KN zb$d+ngBW(WBZ3H{*1sJVmcSw(ikOsJc{gjy9&?5J`LsUKm0m#Lkslc#c2Ge~EX$0bzXjb~w-mIPEfUaFVc?qh@qB!qZcxeH0C8QCrt_iq5!HHH z6V|YTK^|-UwGQdU!lmLvb;SDN9mHbK=SO_a);!FBMx22bfoEtxqr=sTc6Y|` zp}?Z046N1cu9PCYyW{-r_HL0$)rgnt8ZcNTTNUQAq9n60*O1ceFpKT-$EDB>O>rqP z>ep{tKlPOC;POU(1BNuq;VZ>Myd)E3{IjtsFF zSvv{Tt0A-R1hY2a;+o?SLE7LiRt8R$z0Ij`(Ka1nSNVs1@bFFjh9jLBH8UigF3h?` zeS>Of!^54~v+}P6->(UHP7KvMm@*b8Pvh3snwwRJl;k&qff)x}^jfHazg8|4ek{ zc@NuONlDdn(mrN+g)w^utIS9Re3z6JGM{+mgxlS+|`wh}c<1t8rwwO{G|3gYgQ@VP6-Y zy5+Ncmt5Akb0g4gm%NB`!gQTX?!|9fekh;Jlj%GW`zuO;bOS3lxciL)qT;C>6`*D5 z@bn+hp5nLZgq;1Kx}i0VN8W-WC-m&IJob58Oyj9WW2aPJCr_FOjO5tm?L459f}vEm zbPJOcbM@&A@fP2)HE-8^!OxR%uZP*bin67iBHA4{i7Yp{BXh<4&G_`qHYX~11#hQD zHn>l0BEA;}#tX*;MStq4_{`rs#rp3=zwoW#M#E1(7#l$IGBPkr_`?ePg=wScj}4f} zYz-frGeU*0EibF0>dqVKXOl15RFV`-d-I8!yi(indNV0uHbeKm zuPGS(lr29E%Q}?sUAnPfDv7Iyw@B*b@v@oeu_NX|$x3etFkN8YnGl#IOIftJQkhG+ zIe)JxO43Vi+e#o-pf9aO2w$U7=d-qs#u9U=zGRv2lDUR;LUB5B@p;#aZ zEp2d1PUwMlfwst8I}mz+XSQlCrA?I`r3;-44kTcIQVLm{(_$`}X6aEWJfN_XaJ%cTdZ@$m32_kmnH^ZX@)4vlYaCW1k9nE9l%c`uaD=|- z3MFqk>K+y8imuhKaYANoaBamvG)H$gAo%usnqAIOS^E4f#MNNq_lBs(*h=2R^k+kd zfslm}#g=fl5`WA^ygaY&{ec&Z0gFCWo_Vh{D3T-*=ezZ2T}W zwiqhQv4Wsry&aQQWEpSkgAigQVA`d8?!>p*P@ZYoLf5_YBK6N?%&J190h)w_viR;Kf7ln=a}}qG-Eq9dx(LUU6r6zsR#l8slgPDAlWLgH?4`V z_@G)$Y%(M13Y+;FR$E?N4e!V=U;pA^F=xt{gzO10kQ_8xi)gn{!H;K|jQ@&dJg-jL z^FYSBsEOaK&IMk@I>1Rrb+x)!Tgxw2y7HKp2Yd+kuU{OY_!oSnKg`#V9WX}tZJ zGMa?si5ijSEb_f1c;mAoxAgP;Irk5Y*0y8INywG*<^C=woJCDr`2fU zE+xUxfU9+p({xcbR5n1Sr3)n~#<2I?lC(IE;>+NXbEvE4qi-^@T?vCnT`$ zOcW&?Kf@xmb=Gd0)P1jrsP~iw0wiAgr3ETH%=NkD)#E=k<tK_qE?g(RHg6R8;b6s;coP7ZQHtD^@`!eAc`2 zWNTG6?tOV{mZ;*J0Ob@qq`;>Abc_da!-^bzlu$m*Qcu*+Ix{-UZ{7^1silA|uu@DP z8fC}1*+f$AdUEP#*chw{rz0L&!J!jyl#B_F;>;Wuj3b~EaFC3@vbyf6sVw_foSv9? z&#Ux1lWJX^cSl?KR;Y+yqP&QCP+#6iD(5^2k9}x=$QfF=H07@khZ4W*bb}sgc2gah zce5zM1X(&*(*BP9^ek(nNl6u+Bib4@#aYJFEoaPC5z*^OJxk+L+@|{Wp`wD=>xb|h zu_GL}-osLGrAC7ohWJK*!bY*(IR`7lZZ=3y?r`SSq~WEQ@%*hp>YZFb|8~2x{p~p2 z7w_4Etl@-%kNRS(#HCv9o zPKF^>f4BCj=Nt2wP>7egYwIFsH!7=Ag(P(0#{tye32%Pf_@r~n@Szu5UtgKv+Z$m$ zQ}L+@@*3)o=1ioDr19SG9l5QUl%p3_QM33c)zmZ35!sCnH?C$3 z*7jJjKf-Ny<-Tvrk0@y~HcyDP8m^}CX14Up;$UQcXiU2?FEZDAu;SZ3zYq-(w~)Y< zVqR`RJ~5SgSMJ!3XSo})B*`Z)WPWSo2gdeHOo%f=(yM7|BsAoJ;?;I`4a(;r`>q1a zZ_)UWhcI)uY`)VME8JPaur1`ii|K(rJLAi!#vy!wL7U(iBNQIjm8Pod!w0 zOH<`4h^#LEl`mWp;rlxAFGU`U5G%XhgKJzPY&soL*O9+v#iQJ~Eyi)2H6t4MV0p(n z8ME|(MvH~@ubn(r?tIzRwciRQZ%1k1%mpJU>02h+pN=8vd2{#nVc9VLYt`)hCm@>NBOxab_J3Qkv#>Jsz|`f1MIQNCQu$hO z_$E^BaH+O53HOeYHMuh+E_O#Oz@B3VS%HDOARS?bL zga~M@$U-Z1d?Tx;xPu;7HxNWY_?6>JQM9)12}iL>BL`#g>H<5(ISSbRnJuIu&&WPM zLCN>)V;In_l|oZ1Q!IWi@yjbbPat9PCt$HD3(elr7-a8jPsa}a^umZw1(sVim8Np) z-5qZLWyj9lAmwcT$a~S&)553@1Pimk-Kf`w=wx;7Xq1mu8=k*BQ{sq6rUA63ri+p z{ItQS+j(EkXMEwkvnA$QTZ|G69cAn8=UXK!~*o?>3Uxp-lBXI=}8HhCYwx8{ZE4iDYu8M>#ha!+GmaDcCGaAZY? zghxXkb^oTeps<#pV4bi~gRq#cyDu>b1|e;_C)L%@c7ou@_&uT4PzQJKo9>=&0L&U67T@C9 zEl70N??jb6EbuV#wpu;r-4yBvSLA4w!#L@ZocH&>i4rsf^$w`biVdE$DyiKq&cxqj z3XW8%OKY?bYhB?cE8KvdMg@fp2u1zeuA5I|dCnmnY0gvp+ZE-icr}YwT85Y7`L$m- z6;1Uv><;%C>~w`fxl#tO^Fq05CLTDs{^I`B_T%AvC@Sw8j_k1B~LGuX7Gl zf7jF(AP0gRc>9P7;6u{X(7npO;0X3XVw~eYyS=%nz4%cv#d#is9qJ-a(q{1@RpnoY zeO@}Bi+NG2V&l$5sg|GF1z%gmUl%yyfE16Brj+|H=E^@1$Qh;`{O9IrCt&AkY2yq9V`~0e@0lR*wwo5~`1V)uR-T6R zzh%q_TI|{|Mhrmc3KbZ>m67#K;!kTON5j@#1 zN&8}>rOgOQfxi(%=fPy(_MRzVfrBo|c|Z>w^%9|JE=B&lM1zNq1~a|qJEtJ(JELGC z0fa0B`2O4a%OT|c<*5z(MFs|xlRu|$Lz{t*8cm#wphau&C&CiA!pHTmBxr_rv>=ch znCm%L6&+535(3%efk0R<0cly$jPmSwfI=??n)T94@|VCKB53eFoC^)S^m6GX;3G+} zsf^&eru1mwpPl!!jrU8yGzB!{`93@o8hmNl@)8)(ywH`D?7v0?G~SHCnS1@24L~5rCGg}!fY_Q7DCr|&J-amjJCy%}tqrra z0GzGqf$csb^hf}lfG%)++sP83?`JyCVH-cwINcIx_oo94#<|dhmqx*V*udY~Fism_ zX+Rte(z!#xR03X&2jJqa>?II66v*@B05(e*fO6!yimhcD!igaGF&Lv(?h=^N9u2PY z!UZ1n;{t^PjOU6y7&}w!)DbLY1d_)%&~^Sh%>I`q*%?jqdYJoMycI3p;QfajyNwKx z1Re3rV5?S>*Ox$TT*1-yEV}bQM{U@QA6g(i1YA%XyD-{u|CuH&+6e`~|G0HI+HKyVBXuo%2B^y$bM%SJe`TflwEan6{4u)(v^ zu4h2jpvrjzv|Qex5iBn$|l6^|^erLm)cK;WSNUcS^6<3S)hpf`LzBLJCbrFZhs z;_7x3z*hfx+m%OcHCq7Vu)wJ>&qvp>ql#tCgx1 zq+kUp_%2bv%7D%hX|$^K9aUbbAm|O~9+!JOvhq(oL59tPOdl*mMn=YP*5~czV&d07ND&7^FD@X{7#;oOKFgBd|NH zjIN8H%03(q1hjy&i0~!QP7|8ZcJ!j#WYOKG)QZjx8acl{F3*Mk%GZX?y#r@PenK*G L2;^bMS;7AUu<72c diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index c0abcf1d..842c8c5a 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ -#Tue Dec 06 22:38:25 EST 2016 +#Mon Apr 01 18:19:43 PDT 2019 distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-3.2.1-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-5.2.1-all.zip diff --git a/gradlew b/gradlew index 9d82f789..4453ccea 100755 --- a/gradlew +++ b/gradlew @@ -1,4 +1,4 @@ -#!/usr/bin/env bash +#!/usr/bin/env sh ############################################################################## ## @@ -6,12 +6,30 @@ ## ############################################################################## -# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -DEFAULT_JVM_OPTS="" +# Attempt to set APP_HOME +# Resolve links: $0 may be a link +PRG="$0" +# Need this for relative symlinks. +while [ -h "$PRG" ] ; do + ls=`ls -ld "$PRG"` + link=`expr "$ls" : '.*-> \(.*\)$'` + if expr "$link" : '/.*' > /dev/null; then + PRG="$link" + else + PRG=`dirname "$PRG"`"/$link" + fi +done +SAVED="`pwd`" +cd "`dirname \"$PRG\"`/" >/dev/null +APP_HOME="`pwd -P`" +cd "$SAVED" >/dev/null APP_NAME="Gradle" APP_BASE_NAME=`basename "$0"` +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS="" + # Use the maximum available, or set MAX_FD != -1 to use that value. MAX_FD="maximum" @@ -30,6 +48,7 @@ die ( ) { cygwin=false msys=false darwin=false +nonstop=false case "`uname`" in CYGWIN* ) cygwin=true @@ -40,26 +59,11 @@ case "`uname`" in MINGW* ) msys=true ;; + NONSTOP* ) + nonstop=true + ;; esac -# Attempt to set APP_HOME -# Resolve links: $0 may be a link -PRG="$0" -# Need this for relative symlinks. -while [ -h "$PRG" ] ; do - ls=`ls -ld "$PRG"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '/.*' > /dev/null; then - PRG="$link" - else - PRG=`dirname "$PRG"`"/$link" - fi -done -SAVED="`pwd`" -cd "`dirname \"$PRG\"`/" >/dev/null -APP_HOME="`pwd -P`" -cd "$SAVED" >/dev/null - CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar # Determine the Java command to use to start the JVM. @@ -85,7 +89,7 @@ location of your Java installation." fi # Increase the maximum file descriptors if we can. -if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then +if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then MAX_FD_LIMIT=`ulimit -H -n` if [ $? -eq 0 ] ; then if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then @@ -150,11 +154,19 @@ if $cygwin ; then esac fi -# Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules -function splitJvmOpts() { - JVM_OPTS=("$@") +# Escape application args +save ( ) { + for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done + echo " " } -eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS -JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" +APP_ARGS=$(save "$@") + +# Collect all arguments for the java command, following the shell quoting and substitution rules +eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" + +# by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong +if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then + cd "$(dirname "$0")" +fi -exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" +exec "$JAVACMD" "$@" diff --git a/gradlew.bat b/gradlew.bat index aec99730..e95643d6 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -8,14 +8,14 @@ @rem Set local scope for the variables with windows NT shell if "%OS%"=="Windows_NT" setlocal -@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -set DEFAULT_JVM_OPTS= - set DIRNAME=%~dp0 if "%DIRNAME%" == "" set DIRNAME=. set APP_BASE_NAME=%~n0 set APP_HOME=%DIRNAME% +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS= + @rem Find java.exe if defined JAVA_HOME goto findJavaFromJavaHome @@ -46,10 +46,9 @@ echo location of your Java installation. goto fail :init -@rem Get command-line arguments, handling Windowz variants +@rem Get command-line arguments, handling Windows variants if not "%OS%" == "Windows_NT" goto win9xME_args -if "%@eval[2+2]" == "4" goto 4NT_args :win9xME_args @rem Slurp the command line arguments. @@ -60,11 +59,6 @@ set _SKIP=2 if "x%~1" == "x" goto execute set CMD_LINE_ARGS=%* -goto execute - -:4NT_args -@rem Get arguments from the 4NT Shell from JP Software -set CMD_LINE_ARGS=%$ :execute @rem Setup the command line diff --git a/semantic-build-versioning.gradle b/semantic-build-versioning.gradle new file mode 100644 index 00000000..e69de29b diff --git a/settings.gradle b/settings.gradle index e69de29b..bbe83634 100644 --- a/settings.gradle +++ b/settings.gradle @@ -0,0 +1,16 @@ + +buildscript { + repositories { + maven { + url 'https://plugins.gradle.org/m2/' + } + } + dependencies { + classpath 'gradle.plugin.net.vivin:gradle-semantic-build-versioning:4.0.0' + } +} + +apply plugin: 'net.vivin.gradle-semantic-build-versioning' + +//otherwise it defaults to the folder name +rootProject.name = 'kafka-monitor' diff --git a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java index 84a011eb..ad3ce1e4 100644 --- a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java @@ -12,7 +12,7 @@ /** * A base consumer used to abstract different consumer classes. * - * Implementations of this class must have constructor with the following signature:
+ * Implementations of this class must have constructor with the following signature: * Constructor({@link java.util.Properties} properties). */ public interface KMBaseConsumer { diff --git a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java index 3f4c59f3..3c4fa524 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java @@ -15,7 +15,7 @@ /** * Constructs the monitor topic if it does not exist. * - * Implementations of this class should have a public constructor with the following signature:
+ * Implementations of this class should have a public constructor with the following signature: * Constructor(Map<String, ?> config) where config are additional configuration parameters passed in from the Kafka * Monitor configuration. */ From 1a083b5600f05acb4d80a9aadee65ee80630d46b Mon Sep 17 00:00:00 2001 From: Xiongqi Wesley Wu Date: Thu, 25 Apr 2019 15:36:05 -0700 Subject: [PATCH 036/192] add bintray account and publication script --- .travis.yml | 34 ++++++++++++++++++++++++++++++++-- scripts/publishToBintray.sh | 22 ++++++++++++++++++++++ 2 files changed, 54 insertions(+), 2 deletions(-) create mode 100755 scripts/publishToBintray.sh diff --git a/.travis.yml b/.travis.yml index 45e9d6b5..bb6c28bb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,4 +1,34 @@ language: java +dist: trusty +jdk: oraclejdk8 -jdk: - - oraclejdk8 +env: + global: + # these secure values are encrypted for linkedin/kafka-monitor. for use in forked repos, set env vars in your own build - they should win over these (as these wont be decodable in forked builds) + - secure: ghCdYRfSGI7BtA0rB0UfzEJWszKwvnSy1Y0LBSWn3wdo3e0ibHdofwfeQ5YAczXun+RpGvaUwJRgkSDOsL6DFF8N2hODNkzR+DFNRJv5CFOYuCPJZwPpw/Sr7blCkpZleWmJBW7sqRPa7Yyu5/QpdezRM4tHP/RCxjr9IKgQwOknYyVGUf81Wa3Y+MF5iqR434WItfFpHR7uU5sis592iR4aKS4bcEQ9DCqcrCsrr3RFAft6aM5NZjAhO7tXe3vu5IrtqbB/MFNLrnRO0T6kXcifVqVrHQ/ePzl2iyPIdKygiDgK9VOYg6TfT5maHLwoCK511qnhMlitsy/0qx8CsF33rS0hXS5+EDSJ3qoB38Rqgi2K08uGjWgwwXtUg+2CkyEDA2gpEDIELQSFvG0WEHmAim9Azfxiiyx+HwNJ0zkKeOPUXU3uLDeOGXsUB6vxqjrWvAGtUqCZ5g2qEI8geGH3Zopmg9j665FywawV0OAehj4I2Tn0rBDORbIsu3XWgcZIbq/D8sb4Xhb65CB4wSvcco5g25zQzXruDk6MjB76bthtnIbO9hvIVwADxx2MdWXOX873ApjRoffGtRev+ugfEZCEuWxBvtgJ2owCSdTOm8TIjAwjsc4eBuLqEWQplwnoGhRA12zch4t/itOGv0ABoT9/tVjhiLwtTzmHTFc= + - secure: I88NyZbbsV4j5a14k1ModOLsQygs7cXjCY98QQcQNRV0FKvvk/ka7xEvwiOFPWFOFy7TUf9O2VjNA8M4oWGqop9MwYNysAccuX6y/VMAldOPlQ23PG7C+Tr44u2YgvbJCdjtmpVKh842yrU8nlHcfLKBGMaqRxjBg4kHNr0lyTGIU83UBsU26yFiUBuE0EwdWXcrlX+ZvfS523BezquWnFW+75R4z2e0i++/vGqKnlBsqXzzxR61aFnYuSN0CDj+XbG1hsc1dcNJbjLVfO8zOSd25oDYVEX+dfBEfD+4LL/3YSjM8EJvUZGghiaDR+K9nm26B+nfvWNOWH8PuqM94t65vueQ4qcUIUXqwhbIf5GixHGTOIj7QOjckPHZ4N4hYkpYMLwPRO+I5VuBEKlboCN90c3+OEEsBSUXSH/JIBI8mmiwkegPV89HtSRntFGFqYtL7Wg/GyJV7DCQSsacJDaM9ErX2z1sUd++QAQA18qdk1Ngl2hnBOWmTgmtefMo9u15iCxDZr+iMeeTA0IvsupyXeO6hN9xFuwRuXD/q40RIYcOgqFeYR9e/pRJ1A29PE4N6uqDe/JBA8oQFH0jCFbO2oEpVJlX9qgp29wWLSvm/93LRK3f8RLt8ZT1BVNEUnRqtkprbDVp2QlN/7fOz9MI9a00SAXv+siaz3oK++k= + +install: true +script: +- "./gradlew clean build" +before_cache: +- rm -rvf $HOME/.gradle/caches/modules-2/metadata* +- rm -rvf $HOME/.gradle/caches/modules-2/modules-2.lock +- rm -rvf $HOME/.gradle/caches/*/fileHashes/fileHashes.lock +- rm -rvf $HOME/.gradle/caches/*/javaCompile/javaCompile.lock +- rm -rvf $HOME/.gradle/caches/journal-*/*.lock +- rm -rvf $HOME/.gradle/caches/journal-*/file-access.bin +- rm -rvf $HOME/.gradle/caches/*/plugin-resolution/ +- rm -rvf $HOME/.gradle/caches/*/scripts/*/cp_proj/ +- sync +cache: + directories: + - "$HOME/.gradle/caches/" + - "$HOME/.gradle/wrapper/" +deploy: + provider: script + script: bash -v scripts/publishToBintray.sh + skip_cleanup: true + on: + all_branches: true + tags: true diff --git a/scripts/publishToBintray.sh b/scripts/publishToBintray.sh new file mode 100755 index 00000000..80ea5ac6 --- /dev/null +++ b/scripts/publishToBintray.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +result=${PWD##*/} +if [[ "$result" = "scripts" ]] +then + echo "script must be run from root project folder, not $PWD" + exit 1 +else + echo "we are in $PWD and tag is $TRAVIS_TAG" + + if [[ $TRAVIS_TAG =~ ^[0-9]+\.[0-9]+\.[0-9]+$ ]] + then + echo "tag $TRAVIS_TAG looks like a semver so proceeding with bintray publish" + git status + git describe --tags + ./gradlew printVersion + ./gradlew bintrayUpload + else + echo "tag $TRAVIS_TAG is NOT a valid semantic version (x.y.z) so not publishing to bintray" + fi +fi + From ad4ea9d7e1bebefa83a082a0bce97db87c362e13 Mon Sep 17 00:00:00 2001 From: Xiongqi Wesley Wu Date: Thu, 25 Apr 2019 20:50:25 -0700 Subject: [PATCH 037/192] add pom license --- build.gradle | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/build.gradle b/build.gradle index 0e947120..ef0795c5 100644 --- a/build.gradle +++ b/build.gradle @@ -98,7 +98,21 @@ allprojects { artifact testJar pom { + name = 'kafka-monitor' description = 'kafka monitor' + url = 'https://github.com/linkedin/kafka-monitor' + + licenses { + license { + name = 'The Apache Software License, Version 2.0' + url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' + } + } + scm { + connection = 'scm:git:git://github.com:linkedin/kafka-monitor.git' + developerConnection = 'scm:git:ssh://github.com:linkedin/kafka-monitor.git' + url = 'https://github.com/linkedin/kafka-monitor' + } } } } From 83c2adad0ec7172102bdaf0ff10afea25b0c6012 Mon Sep 17 00:00:00 2001 From: Xiongqi Wu Date: Tue, 14 May 2019 11:31:44 -0700 Subject: [PATCH 038/192] allow brokers to be blacklisted when adding new partitions or reassigning partitions (#154) KMF today assign partitions or add new partitions to all kafka brokers in a cluster. However, there are use cases that we don't want KMF topic partitions to be created on certain brokers. For example, broker that is not in a healthy state and is being removed might not want to take any new KMF partitions. This fix allows user to blacklist certain kafka brokers via topic factory so that blacklisted brokers will not be assigned new partitions or taking reassigned partitions. The default topic factory implementation doesn't blacklist any brokers. User can have custom implementation of topic factory where blacklisted broker info can be obtained dynamically. --- .../MultiClusterTopicManagementService.java | 26 ++++++++++++++++--- .../kmf/topicfactory/DefaultTopicFactory.java | 7 +++++ .../kmf/topicfactory/TopicFactory.java | 9 ++++++- 3 files changed, 38 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 90ac42c8..4a76b0af 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -265,23 +265,43 @@ void maybeAddPartitions(int minPartitionNum) { if (partitionNum < minPartitionNum) { LOG.info("MultiClusterTopicManagementService will increase partition of the topic {} " + "in cluster {} from {} to {}.", _topic, _zkConnect, partitionNum, minPartitionNum); - + Set blackListedBrokers = + _topicFactory.getBlackListedBrokers(_zkConnect); scala.Option>> replicaAssignment = scala.Option.apply(null); scala.Option> brokerList = scala.Option.apply(null); - adminZkClient.addPartitions(_topic, existingAssignment, adminZkClient.getBrokerMetadatas(RackAwareMode.Disabled$.MODULE$, brokerList), minPartitionNum, replicaAssignment, false); + Set brokers = + new HashSet<>(scala.collection.JavaConversions.asJavaCollection(adminZkClient.getBrokerMetadatas(RackAwareMode.Disabled$.MODULE$, brokerList))); + + if (!blackListedBrokers.isEmpty()) { + brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); + } + adminZkClient.addPartitions(_topic, existingAssignment, + scala.collection.JavaConversions.collectionAsScalaIterable(brokers).toSeq(), + minPartitionNum, replicaAssignment, false); } } finally { zkClient.close(); } } + private Set getAvailableBrokers(KafkaZkClient zkClient) { + Set brokers = + new HashSet<>(scala.collection.JavaConversions.asJavaCollection(zkClient.getAllBrokersInCluster())); + + Set blackListedBrokers = + _topicFactory.getBlackListedBrokers(_zkConnect); + + brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); + return brokers; + } + void maybeReassignPartitionAndElectLeader() throws Exception { KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); try { List partitionInfoList = getPartitionInfo(zkClient, _topic); - Collection brokers = scala.collection.JavaConversions.asJavaCollection(zkClient.getAllBrokersInCluster()); + Collection brokers = getAvailableBrokers(zkClient); boolean partitionReassigned = false; if (partitionInfoList.size() == 0) throw new IllegalStateException("Topic " + _topic + " does not exist in cluster " + _zkConnect); diff --git a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java index 82325e41..5ca24acf 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java @@ -11,8 +11,10 @@ import com.linkedin.kmf.common.Utils; +import java.util.Collections; import java.util.Map; import java.util.Properties; +import java.util.Set; public class DefaultTopicFactory implements TopicFactory { @@ -25,4 +27,9 @@ public DefaultTopicFactory(Map config) { public int createTopicIfNotExist(String zkUrl, String topic, int replicationFactor, double partitionToBrokerRatio, Properties topicConfig) { return Utils.createTopicIfNotExists(zkUrl, topic, replicationFactor, partitionToBrokerRatio, 1, topicConfig); } + + @Override + public Set getBlackListedBrokers(String zkUrl) { + return Collections.emptySet(); + } } diff --git a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java index 3c4fa524..54f1dc30 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java @@ -10,10 +10,11 @@ package com.linkedin.kmf.topicfactory; import java.util.Properties; +import java.util.Set; /** - * Constructs the monitor topic if it does not exist. + * Constructs the monitor topic if it does not exist, and provide blacklisted brokers info for topic management service * * Implementations of this class should have a public constructor with the following signature: * Constructor(Map<String, ?> config) where config are additional configuration parameters passed in from the Kafka @@ -34,4 +35,10 @@ public interface TopicFactory { int createTopicIfNotExist(String zkUrl, String topic, int replicationFactor, double partitionToBrokerRatio, Properties topicProperties); + /** + * @param zkUrl zookeeper connection url + * @return A set of brokers that don't take new partitions or reassigned partitions for topics. + */ + Set getBlackListedBrokers(String zkUrl); + } From 96b0b3be2b399e708248cd2880cea6bc12765af5 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 6 Nov 2019 15:10:52 -0800 Subject: [PATCH 039/192] Addition of Tracelogs for recordsLost in Consume Service (#163) * Addition of tracelogs for recordsLost in Consume Service to indicate record index and the next index. --- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index f55193e5..6352320b 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -173,7 +173,9 @@ record = _consumer.receive(); _sensors._recordsDuplicated.record(); } else if (index > nextIndex) { nextIndexes.put(partition, index + 1); - _sensors._recordsLost.record(index - nextIndex); + long numLostRecords = index - nextIndex; + _sensors._recordsLost.record(numLostRecords); + LOG.info("_recordsLost recorded: Avro record current index: {} at {}. Next index: {}. Lost {} records.", index, currMs, nextIndex, numLostRecords); } } } From eca60c7ec20f199ad1a8452021d4186edacee0d2 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 18 Nov 2019 16:16:28 -0800 Subject: [PATCH 040/192] Addition of lost rate / lost record as well as Log divider for separation --- config/kafka-monitor.properties | 1 + config/multi-cluster-monitor.properties | 3 ++- .../java/com/linkedin/kmf/KafkaMonitor.java | 2 +- .../kmf/apps/SingleClusterMonitor.java | 3 ++- .../linkedin/kmf/services/ConsumeService.java | 6 ++--- .../DefaultMetricsReporterService.java | 24 ++++++++----------- 6 files changed, 19 insertions(+), 20 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index cc2e13dd..0800ad99 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -88,6 +88,7 @@ "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", "kmf.services:type=consume-service,name=*:records-lost-total", + "kmf.services:type=consume-service,name=*:records-lost-rate", "kmf.services:type=consume-service,name=*:records-duplicated-total", "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", diff --git a/config/multi-cluster-monitor.properties b/config/multi-cluster-monitor.properties index a236503d..cc1ce37a 100644 --- a/config/multi-cluster-monitor.properties +++ b/config/multi-cluster-monitor.properties @@ -47,7 +47,7 @@ "topic-management.topicFactory.props": { } }, - + # If there are more than two clusters in the pipeline, # add one property map for each one of them. @@ -73,6 +73,7 @@ "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", "kmf.services:type=consume-service,name=*:records-lost-total", + "kmf.services:type=consume-service,name=*:records-lost-rate", "kmf.services:type=consume-service,name=*:records-duplicated-total", "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index e2f93d99..3f6e509c 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -174,7 +174,7 @@ public static void main(String[] args) throws Exception { Map props = new ObjectMapper().readValue(buffer.toString(), Map.class); KafkaMonitor kafkaMonitor = new KafkaMonitor(props); kafkaMonitor.start(); - LOG.info("KafkaMonitor started"); + LOG.info("KafkaMonitor started."); kafkaMonitor.awaitShutdown(); } diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index d118930f..74ef6cf0 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -284,6 +284,7 @@ public static void main(String[] args) throws Exception { "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", "kmf.services:type=consume-service,name=*:records-lost-total", + "kmf.services:type=consume-service,name=*:records-lost-rate", "kmf.services:type=consume-service,name=*:records-duplicated-total", "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", @@ -301,7 +302,7 @@ public static void main(String[] args) throws Exception { jettyService.start(); if (!app.isRunning()) { - LOG.error("Some services have stopped"); + LOG.error("Some services have stopped."); System.exit(-1); } app.awaitShutdown(); diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 6352320b..5b447f54 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -184,7 +184,7 @@ record = _consumer.receive(); public synchronized void start() { if (_running.compareAndSet(false, true)) { _thread.start(); - LOG.info("{}/ConsumeService started", _name); + LOG.info("{}/ConsumeService started.", _name); } } @@ -196,13 +196,13 @@ public synchronized void stop() { } catch (Exception e) { LOG.warn(_name + "/ConsumeService while trying to close consumer.", e); } - LOG.info("{}/ConsumeService stopped", _name); + LOG.info("{}/ConsumeService stopped.", _name); } } @Override public void awaitShutdown() { - LOG.info("{}/ConsumeService shutdown completed", _name); + LOG.info("{}/ConsumeService shutdown completed.", _name); } @Override diff --git a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java index b6f8dac3..b01500ad 100644 --- a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java @@ -23,6 +23,7 @@ public class DefaultMetricsReporterService implements Service { private static final Logger LOG = LoggerFactory.getLogger(DefaultMetricsReporterService.class); + private static final String LOG_DIVIDER = "============================================================="; private final String _name; private final List _metricNames; @@ -39,18 +40,13 @@ public DefaultMetricsReporterService(Map props, String name) { @Override public synchronized void start() { - _executor.scheduleAtFixedRate( - new Runnable() { - @Override - public void run() { - try { - reportMetrics(); - } catch (Exception e) { - LOG.error(_name + "/DefaultMetricsReporterService failed to report metrics", e); - } - } - }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS - ); + _executor.scheduleAtFixedRate(() -> { + try { + reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/DefaultMetricsReporterService failed to report metrics.", e); + } + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); LOG.info("{}/DefaultMetricsReporterService started", _name); } @@ -72,7 +68,7 @@ public void awaitShutdown() { } catch (InterruptedException e) { LOG.info("Thread interrupted when waiting for {}/DefaultMetricsReporterService to shutdown", _name); } - LOG.info("{}/DefaultMetricsReporterService shutdown completed", _name); + LOG.info("{}/DefaultMetricsReporterService shutdown completed.", _name); } private void reportMetrics() { @@ -86,6 +82,6 @@ private void reportMetrics() { builder.append("\n"); } } - LOG.info(builder.toString()); + LOG.info("{}\n{}", LOG_DIVIDER, builder.toString()); } } From 75bff244e87079f98a5f65f17ceb4195d8b29ca6 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 21 Nov 2019 12:42:23 -0800 Subject: [PATCH 041/192] READ ME update --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 595c0f9d..f72294c2 100644 --- a/README.md +++ b/README.md @@ -111,7 +111,7 @@ $ ./bin/kafka-monitor-start.sh config/multi-cluster-monitor.properties ``` ### Get metric values (e.g. service availability, message loss rate) in real-time as time series graphs -Open ```localhost:8000/index.html``` in your web browser +Open ```localhost:8000/index.html``` in your web browser. You can edit webapp/index.html to easily add new metrics to be displayed. From b257322621675e3beab3db0e1ba936ac7b324338 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 21 Nov 2019 12:44:04 -0800 Subject: [PATCH 042/192] comments whitespacing update --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index ef0795c5..9b6a0607 100644 --- a/build.gradle +++ b/build.gradle @@ -56,7 +56,7 @@ allprojects { jar { doFirst { manifest { - //embed version information into jar manifests + // embed version information into jar manifests attributes('Name': "${project.name}", 'Specification-Title': "${project.name}", 'Specification-Version': "${project.version}", @@ -149,7 +149,7 @@ bintray { repo = 'maven' name = 'kafka-monitor' userOrg = 'linkedin' - licenses = ['Apache-2.0'] //this is validated by bintray and must be a valid license + licenses = ['Apache-2.0'] // this is validated by bintray and must be a valid license vcsUrl = 'https://github.com/linkedin/kafka-monitor.git' version { name = project.version From 17c8d09d82f009b67225d83c6677db28c0ece10d Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 22 Nov 2019 11:55:22 -0800 Subject: [PATCH 043/192] settings gradle org clean up --- settings.gradle | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/settings.gradle b/settings.gradle index bbe83634..f37e0228 100644 --- a/settings.gradle +++ b/settings.gradle @@ -1,4 +1,3 @@ - buildscript { repositories { maven { @@ -12,5 +11,5 @@ buildscript { apply plugin: 'net.vivin.gradle-semantic-build-versioning' -//otherwise it defaults to the folder name +// otherwise it defaults to the folder name rootProject.name = 'kafka-monitor' From 49d7fe3d8b04e789438beee86fbd583d24f43471 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 22 Nov 2019 16:17:11 -0800 Subject: [PATCH 044/192] styling for b/gradlew files --- gradlew | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/gradlew b/gradlew index 4453ccea..2477741a 100755 --- a/gradlew +++ b/gradlew @@ -104,18 +104,18 @@ if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then fi fi -# For Darwin, add options to specify how the application appears in the dock +# For Darwin, add options to specify how the application appears in the dock. if $darwin; then GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" fi -# For Cygwin, switch paths to Windows format before running java +# For Cygwin, switch paths to Windows format before running java. if $cygwin ; then APP_HOME=`cygpath --path --mixed "$APP_HOME"` CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` JAVACMD=`cygpath --unix "$JAVACMD"` - # We build the pattern for arguments to be converted via cygpath + # We build the pattern for arguments to be converted via cygpath. ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` SEP="" for dir in $ROOTDIRSRAW ; do @@ -123,7 +123,7 @@ if $cygwin ; then SEP="|" done OURCYGPATTERN="(^($ROOTDIRS))" - # Add a user-defined pattern to the cygpath arguments + # Add a user-defined pattern to the cygpath arguments. if [ "$GRADLE_CYGPATTERN" != "" ] ; then OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" fi From 42eb84843a0045113326d5a7ee29e8bbb2b81b9b Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sat, 23 Nov 2019 17:26:36 -0800 Subject: [PATCH 045/192] Add license and notice --- LICENSE | 1 - 1 file changed, 1 deletion(-) diff --git a/LICENSE b/LICENSE index 02c5bb4d..1e031347 100644 --- a/LICENSE +++ b/LICENSE @@ -1,4 +1,3 @@ - Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ From eee7983df861ed1c1cc4a7686af291d90e5d9dd7 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 25 Nov 2019 10:55:46 -0800 Subject: [PATCH 046/192] Add LICENSE and NOTICE --- NOTICE | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/NOTICE b/NOTICE index ee9a4345..57dcd9c0 100644 --- a/NOTICE +++ b/NOTICE @@ -34,18 +34,3 @@ License: http://www.json.org/license.html This product includes/uses JUnit (https://http://junit.org/) Copyright 2002-2016 JUnit License: Eclipse Public License 1.0 - - - - - - - - - - - - - - - From 6403e638e423969c04004876073cff23b1be1aec Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 Nov 2019 10:52:39 -0800 Subject: [PATCH 047/192] Migration from ZkClient to AdminClient for Multi-Cluster Topic Management Service (#166) * replace zkClient with AdminClient for partition additions * remove unsued method * Use of AdminClientsConfigs * updated with sean's comments * remove adminClientProperties.put(METRIC_GROUP_NAME); * triggerPLE use adminclient to electPLs. * remove redundant import * remove SSL configs * clean up * enable logging for metrics reporter service again * READ ME update * comments whitespacing update * addressed some of the comments * ssl configurations for multi cluster topic management service * keystore truststore config locations and password * wip * settings gradle org clean up * removed SSL * removign scala one by one * styling for b/gradlew files * replace zkClient with AdminClient for partition additions * remove unsued method * Use of AdminClientsConfigs * updated with sean's comments * remove adminClientProperties.put(METRIC_GROUP_NAME); * triggerPLE use adminclient to electPLs. * remove redundant import * remove SSL configs * clean up * enable logging for metrics reporter service again * addressed some of the comments * ssl configurations for multi cluster topic management service * keystore truststore config locations and password * wip * removed SSL * removign scala one by one * wip * wip * wip * testing lowtotalnumberofpartitions fix * remove consumerProps.put("zookeeper.connect", zkConnect * reassign partitions private method in progress * use zkclient for reassignpartitions * bump up dependency on apahce kafka open source * exception handling for describe * completable future usage for race condition handling * replicaassignment initialized with arraylist of arraylist * Usage of completableFutures only on the topicManagementService * remove return thats unused * consumeService revert * addressing final comments * very minor --- build.gradle | 5 +- config/kafka-monitor.properties | 3 +- .../java/com/linkedin/kmf/KafkaMonitor.java | 15 +- .../kmf/apps/SingleClusterMonitor.java | 34 +-- .../java/com/linkedin/kmf/common/Utils.java | 2 +- .../DefaultMetricsReporterService.java | 16 +- .../GraphiteMetricsReporterService.java | 17 +- .../linkedin/kmf/services/JettyService.java | 3 +- .../linkedin/kmf/services/JolokiaService.java | 5 +- .../services/KafkaMetricsReporterService.java | 15 +- .../MultiClusterTopicManagementService.java | 266 +++++++++--------- .../linkedin/kmf/services/ProduceService.java | 87 +++--- .../SignalFxMetricsReporterService.java | 20 +- .../StatsdMetricsReporterService.java | 10 +- .../kmf/services/TopicManagementService.java | 6 + .../linkedin/kmf/tests/BasicEndToEndTest.java | 19 +- .../com/linkedin/kmf/KafkaMonitorTest.java | 29 +- .../services/TopicManagementServiceTest.java | 56 ++-- 18 files changed, 307 insertions(+), 301 deletions(-) diff --git a/build.gradle b/build.gradle index 9b6a0607..e57b22ee 100644 --- a/build.gradle +++ b/build.gradle @@ -30,8 +30,6 @@ allprojects { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.slf4j:slf4j-log4j12:1.7.6' compile 'org.apache.avro:avro:1.4.0' - compile 'org.apache.kafka:kafka_2.12:2.0.0' - compile 'org.apache.kafka:kafka-clients:2.0.0' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' @@ -40,7 +38,8 @@ allprojects { compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' - + compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.3.1' testCompile 'org.testng:testng:6.8.8' } diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 0800ad99..540f24a3 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -46,6 +46,7 @@ "topic": "kafka-monitor-topic", "zookeeper.connect": "localhost:2181", "bootstrap.servers": "localhost:9092", + "request.timeout.ms": 9000, "produce.record.delay.ms": 100, "topic-management.topicCreationEnabled": true, "topic-management.replicationFactor" : 1, @@ -63,9 +64,7 @@ "consume.latency.sla.ms": "20000", "consume.consumer.props": { - } - }, "jetty-service": { diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index 3f6e509c..9fdcdda0 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -10,13 +10,18 @@ package com.linkedin.kmf; import com.fasterxml.jackson.databind.ObjectMapper; -import com.linkedin.kmf.services.Service; import com.linkedin.kmf.apps.App; - +import com.linkedin.kmf.services.Service; +import java.io.BufferedReader; +import java.io.FileReader; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.Measurable; @@ -26,12 +31,6 @@ import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedReader; -import java.io.FileReader; -import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; /** diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 74ef6cf0..4eb4c5cc 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -9,28 +9,28 @@ */ package com.linkedin.kmf.apps; +import com.linkedin.kmf.services.ConsumeService; +import com.linkedin.kmf.services.DefaultMetricsReporterService; +import com.linkedin.kmf.services.JettyService; +import com.linkedin.kmf.services.JolokiaService; +import com.linkedin.kmf.services.ProduceService; import com.linkedin.kmf.services.TopicManagementService; import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import com.linkedin.kmf.services.configs.DefaultMetricsReporterServiceConfig; import com.linkedin.kmf.services.configs.MultiClusterTopicManagementServiceConfig; import com.linkedin.kmf.services.configs.ProduceServiceConfig; -import com.linkedin.kmf.services.ConsumeService; -import com.linkedin.kmf.services.JettyService; -import com.linkedin.kmf.services.JolokiaService; -import com.linkedin.kmf.services.DefaultMetricsReporterService; -import com.linkedin.kmf.services.ProduceService; import com.linkedin.kmf.services.configs.TopicManagementServiceConfig; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.kafka.common.utils.Utils; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import static net.sourceforge.argparse4j.impl.Arguments.store; @@ -61,9 +61,12 @@ public SingleClusterMonitor(Map props, String name) throws Excep @Override public void start() { _topicManagementService.start(); - _produceService.start(); - _consumeService.start(); - LOG.info(_name + "/SingleClusterMonitor started"); + CompletableFuture completableFuture = _topicManagementService.topicManagementReady(); + completableFuture.thenRun(() -> { + _produceService.start(); + _consumeService.start(); + }); + LOG.info(_name + "/SingleClusterMonitor started."); } @Override @@ -71,7 +74,7 @@ public void stop() { _topicManagementService.stop(); _produceService.stop(); _consumeService.stop(); - LOG.info(_name + "/SingleClusterMonitor stopped"); + LOG.info(_name + "/SingleClusterMonitor stopped."); } @Override @@ -234,7 +237,6 @@ public static void main(String[] args) throws Exception { Namespace res = parser.parseArgs(args); Map props = new HashMap<>(); - // produce service config props.put(ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG, res.getString("zkConnect")); props.put(ProduceServiceConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList")); diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 5399f99a..ab8ce2fb 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -96,7 +96,7 @@ public static int createTopicIfNotExists(String zkUrl, String topic, int replica try { AdminUtils.createTopic(zkUtils, topic, partitionCount, replicationFactor, topicConfig, RackAwareMode.Enforced$.MODULE$); } catch (TopicExistsException e) { - //There is a race condition with the consumer. + // There is a race condition with the consumer. LOG.debug("Monitoring topic " + topic + " already exists in cluster " + zkUrl, e); return getPartitionNumForTopic(zkUrl, topic); } diff --git a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java index b01500ad..8345209d 100644 --- a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java @@ -9,21 +9,19 @@ */ package com.linkedin.kmf.services; -import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; - import com.linkedin.kmf.common.MbeanAttributeValue; import com.linkedin.kmf.services.configs.DefaultMetricsReporterServiceConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DefaultMetricsReporterService implements Service { private static final Logger LOG = LoggerFactory.getLogger(DefaultMetricsReporterService.class); - private static final String LOG_DIVIDER = "============================================================="; + private static final String LOG_DIVIDER = "=============================================================="; private final String _name; private final List _metricNames; @@ -47,13 +45,13 @@ public synchronized void start() { LOG.error(_name + "/DefaultMetricsReporterService failed to report metrics.", e); } }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); - LOG.info("{}/DefaultMetricsReporterService started", _name); + LOG.info("{}/DefaultMetricsReporterService started.", _name); } @Override public synchronized void stop() { _executor.shutdown(); - LOG.info("{}/DefaultMetricsReporterService stopped", _name); + LOG.info("{}/DefaultMetricsReporterService stopped.", _name); } @Override @@ -66,7 +64,7 @@ public void awaitShutdown() { try { _executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.info("Thread interrupted when waiting for {}/DefaultMetricsReporterService to shutdown", _name); + LOG.info("Thread interrupted when waiting for {}/DefaultMetricsReporterService to shutdown.", _name); } LOG.info("{}/DefaultMetricsReporterService shutdown completed.", _name); } @@ -76,7 +74,7 @@ private void reportMetrics() { for (String metricName: _metricNames) { String mbeanExpr = metricName.substring(0, metricName.lastIndexOf(":")); String attributeExpr = metricName.substring(metricName.lastIndexOf(":") + 1); - List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + List attributeValues = com.linkedin.kmf.common.Utils.getMBeanAttributeValues(mbeanExpr, attributeExpr); for (MbeanAttributeValue attributeValue: attributeValues) { builder.append(attributeValue.toString()); builder.append("\n"); diff --git a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java index 706defd5..bb3f72b9 100644 --- a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java @@ -10,17 +10,8 @@ package com.linkedin.kmf.services; -import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; - import com.linkedin.kmf.common.MbeanAttributeValue; import com.linkedin.kmf.services.configs.GraphiteMetricsReporterServiceConfig; -import net.savantly.graphite.GraphiteClient; -import net.savantly.graphite.GraphiteClientFactory; -import net.savantly.graphite.impl.SimpleCarbonMetric; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.net.SocketException; import java.net.UnknownHostException; import java.util.List; @@ -28,6 +19,12 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import net.savantly.graphite.GraphiteClient; +import net.savantly.graphite.GraphiteClientFactory; +import net.savantly.graphite.impl.SimpleCarbonMetric; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class GraphiteMetricsReporterService implements Service { private static final Logger LOG = LoggerFactory.getLogger(GraphiteMetricsReporterService.class); @@ -104,7 +101,7 @@ private void reportMetrics() { for (String metricName: _metricNames) { String mbeanExpr = metricName.substring(0, metricName.lastIndexOf(":")); String attributeExpr = metricName.substring(metricName.lastIndexOf(":") + 1); - List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + List attributeValues = com.linkedin.kmf.common.Utils.getMBeanAttributeValues(mbeanExpr, attributeExpr); for (MbeanAttributeValue attributeValue: attributeValues) { _graphiteClient.saveCarbonMetrics( new SimpleCarbonMetric( diff --git a/src/main/java/com/linkedin/kmf/services/JettyService.java b/src/main/java/com/linkedin/kmf/services/JettyService.java index 42b74ae5..e8cc1f25 100644 --- a/src/main/java/com/linkedin/kmf/services/JettyService.java +++ b/src/main/java/com/linkedin/kmf/services/JettyService.java @@ -10,13 +10,12 @@ package com.linkedin.kmf.services; import com.linkedin.kmf.services.configs.JettyServiceConfig; +import java.util.Map; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.ResourceHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; - // Jetty server that serves html files. public class JettyService implements Service { private static final Logger LOG = LoggerFactory.getLogger(JettyService.class); diff --git a/src/main/java/com/linkedin/kmf/services/JolokiaService.java b/src/main/java/com/linkedin/kmf/services/JolokiaService.java index 5218cb32..360ac2ab 100644 --- a/src/main/java/com/linkedin/kmf/services/JolokiaService.java +++ b/src/main/java/com/linkedin/kmf/services/JolokiaService.java @@ -9,14 +9,13 @@ */ package com.linkedin.kmf.services; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import org.jolokia.jvmagent.JolokiaServer; import org.jolokia.jvmagent.JvmAgentConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Jolokia server allows user to query jmx metric value with HTTP request */ diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index d0fd0ecf..1294ead7 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -9,19 +9,11 @@ */ package com.linkedin.kmf.services; -import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; - import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.kmf.common.MbeanAttributeValue; import com.linkedin.kmf.common.Utils; import com.linkedin.kmf.services.configs.KafkaMetricsReporterServiceConfig; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.HashMap; import java.util.List; import java.util.Map; @@ -29,6 +21,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class KafkaMetricsReporterService implements Service { @@ -121,7 +118,7 @@ private void reportMetrics() { for (String metricName : _metricsNames) { String mbeanExpr = metricName.substring(0, metricName.lastIndexOf(":")); String attributeExpr = metricName.substring(metricName.lastIndexOf(":") + 1); - List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + List attributeValues = com.linkedin.kmf.common.Utils.getMBeanAttributeValues(mbeanExpr, attributeExpr); for (MbeanAttributeValue attributeValue : attributeValues) { String metric = attributeValue.toString(); String key = metric.substring(0, metric.lastIndexOf("=")); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 4a76b0af..41cb42a3 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -10,7 +10,6 @@ package com.linkedin.kmf.services; -import com.linkedin.kmf.common.Utils; import com.linkedin.kmf.services.configs.CommonServiceConfig; import com.linkedin.kmf.services.configs.MultiClusterTopicManagementServiceConfig; import com.linkedin.kmf.services.configs.TopicManagementServiceConfig; @@ -18,39 +17,44 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.HashSet; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import kafka.admin.AdminOperationException; import java.util.concurrent.atomic.AtomicBoolean; import kafka.admin.AdminUtils; import kafka.admin.BrokerMetadata; -import kafka.admin.PreferredReplicaLeaderElectionCommand; -import kafka.admin.RackAwareMode; -import kafka.cluster.Broker; import kafka.server.ConfigType; -import kafka.zk.AdminZkClient; import kafka.zk.KafkaZkClient; -import org.I0Itec.zkclient.exception.ZkNodeExistsException; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.CreateTopicsResult; +import org.apache.kafka.clients.admin.ElectPreferredLeadersResult; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.security.JaasUtils; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Option$; import scala.collection.Seq; -import static com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS; -import static com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS; /** * This service periodically checks and rebalances the monitor topics across a pipeline of Kafka clusters so that @@ -63,19 +67,19 @@ * - Increase replication factor of the monitor topic if the user-specified replicationFactor is not satisfied * - Reassign partition across brokers to make sure each broker acts as preferred leader of at least one partition of the monitor topic * - Trigger preferred leader election to make sure each broker acts as leader of at least one partition of the monitor topic. - * - Make sure the number of partitions of the monitor topic is same across all monitored custers. + * - Make sure the number of partitions of the monitor topic is same across all monitored clusters. * */ public class MultiClusterTopicManagementService implements Service { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); private static final String METRIC_GROUP_NAME = "topic-management-service"; - private final AtomicBoolean _isRunning = new AtomicBoolean(false); private final String _serviceName; private final Map _topicManagementByCluster; private final int _scheduleIntervalMs; private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; + final private CompletableFuture _completableFuture; public MultiClusterTopicManagementService(Map props, String serviceName) throws Exception { _serviceName = serviceName; @@ -86,6 +90,7 @@ public MultiClusterTopicManagementService(Map props, String serv _topicManagementByCluster = initializeTopicManagementHelper(propsByCluster, topic); _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); + _completableFuture = new CompletableFuture<>(); _executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { @Override public Thread newThread(Runnable r) { @@ -94,6 +99,10 @@ public Thread newThread(Runnable r) { }); } + public CompletableFuture topicManagementReady() { + return _completableFuture; + } + private Map initializeTopicManagementHelper(Map propsByCluster, String topic) throws Exception { Map topicManagementByCluster = new HashMap<>(); for (Map.Entry entry: propsByCluster.entrySet()) { @@ -113,6 +122,7 @@ public synchronized void start() { if (_isRunning.compareAndSet(false, true)) { Runnable tmRunnable = new TopicManagementRunnable(); _executor.scheduleWithFixedDelay(tmRunnable, 0, _scheduleIntervalMs, TimeUnit.MILLISECONDS); + Runnable pleRunnable = new PreferredLeaderElectionRunnable(); _executor.scheduleWithFixedDelay(pleRunnable, _preferredLeaderElectionIntervalMs, _preferredLeaderElectionIntervalMs, TimeUnit.MILLISECONDS); @@ -144,6 +154,7 @@ public void awaitShutdown() { } private class TopicManagementRunnable implements Runnable { + @Override public void run() { try { @@ -165,13 +176,14 @@ public void run() { for (TopicManagementHelper helper : _topicManagementByCluster.values()) { helper.maybeAddPartitions(minPartitionNum); } + _completableFuture.complete(null); for (Map.Entry entry : _topicManagementByCluster.entrySet()) { String clusterName = entry.getKey(); TopicManagementHelper helper = entry.getValue(); try { helper.maybeReassignPartitionAndElectLeader(); - } catch (IOException | ZkNodeExistsException | AdminOperationException e) { + } catch (IOException | KafkaException e) { LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); } } @@ -184,8 +196,10 @@ public void run() { } } - // Check if Preferred leader election is requested during Topic Management (TopicManagementRunnable), - // trigger Preferred leader election when there is no partition reassignment in progress. + /** + * Check if Preferred leader election is requested during Topic Management (TopicManagementRunnable), + * trigger Preferred leader election when there is no partition reassignment in progress. + */ private class PreferredLeaderElectionRunnable implements Runnable { @Override public void run() { @@ -195,13 +209,13 @@ public void run() { TopicManagementHelper helper = entry.getValue(); try { helper.maybeElectLeader(); - } catch (IOException | ZkNodeExistsException | AdminOperationException e) { + } catch (IOException | KafkaException e) { LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); } } } catch (Throwable t) { - // Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime - // and such error is not caught by compilation + /* Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime + and such error is not caught by compilation. */ LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); stop(); } @@ -209,7 +223,6 @@ public void run() { } static class TopicManagementHelper { - private final boolean _topicCreationEnabled; private final String _topic; private final String _zkConnect; @@ -219,9 +232,14 @@ static class TopicManagementHelper { private final TopicFactory _topicFactory; private final Properties _topicProperties; private boolean _preferredLeaderElectionRequested; + private int _requestTimeoutMs; + private List _bootstrapServers; + private final AdminClient _adminClient; TopicManagementHelper(Map props) throws Exception { TopicManagementServiceConfig config = new TopicManagementServiceConfig(props); + AdminClientConfig adminClientConfig = new AdminClientConfig(props); + String topicFactoryClassName = config.getString(TopicManagementServiceConfig.TOPIC_FACTORY_CLASS_CONFIG); _topicCreationEnabled = config.getBoolean(TopicManagementServiceConfig.TOPIC_CREATION_ENABLED_CONFIG); _topic = config.getString(TopicManagementServiceConfig.TOPIC_CONFIG); _zkConnect = config.getString(TopicManagementServiceConfig.ZOOKEEPER_CONNECT_CONFIG); @@ -229,65 +247,76 @@ static class TopicManagementHelper { _minPartitionsToBrokersRatio = config.getDouble(TopicManagementServiceConfig.PARTITIONS_TO_BROKERS_RATIO_CONFIG); _minPartitionNum = config.getInt(TopicManagementServiceConfig.MIN_PARTITION_NUM_CONFIG); _preferredLeaderElectionRequested = false; - String topicFactoryClassName = config.getString(TopicManagementServiceConfig.TOPIC_FACTORY_CLASS_CONFIG); - + _requestTimeoutMs = adminClientConfig.getInt(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG); + _bootstrapServers = adminClientConfig.getList(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG); _topicProperties = new Properties(); if (props.containsKey(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)) { for (Map.Entry entry: ((Map) props.get(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)).entrySet()) _topicProperties.put(entry.getKey(), entry.getValue().toString()); } - Map topicFactoryConfig = props.containsKey(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) ? (Map) props.get(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) : new HashMap(); _topicFactory = (TopicFactory) Class.forName(topicFactoryClassName).getConstructor(Map.class).newInstance(topicFactoryConfig); + _adminClient = constructAdminClient(props); } void maybeCreateTopic() throws Exception { if (_topicCreationEnabled) { - _topicFactory.createTopicIfNotExist(_zkConnect, _topic, _replicationFactor, _minPartitionsToBrokersRatio, _topicProperties); + int brokerCount = _adminClient.describeCluster().nodes().get().size(); + int numPartitions = Math.max((int) Math.ceil(brokerCount * _minPartitionsToBrokersRatio), minPartitionNum()); + NewTopic newTopic = new NewTopic(_topic, numPartitions, (short) _replicationFactor); + newTopic.configs((Map) _topicProperties); + CreateTopicsResult createTopicsResult = _adminClient.createTopics(Collections.singletonList(newTopic)); + LOG.info("CreateTopicsResult: {}.", createTopicsResult.values()); } } - int minPartitionNum() { - int brokerCount = Utils.getBrokerCount(_zkConnect); + public AdminClient constructAdminClient(Map props) { + props.putIfAbsent(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, _bootstrapServers); + props.putIfAbsent(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, _requestTimeoutMs); + return AdminClient.create(props); + } + + int minPartitionNum() throws InterruptedException, ExecutionException { + int brokerCount = _adminClient.describeCluster().nodes().get().size(); return Math.max((int) Math.ceil(_minPartitionsToBrokersRatio * brokerCount), _minPartitionNum); } - void maybeAddPartitions(int minPartitionNum) { - KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, - ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); - AdminZkClient adminZkClient = new AdminZkClient(zkClient); + void maybeAddPartitions(int minPartitionNum) throws ExecutionException, InterruptedException { + Collection topicNames = _adminClient.listTopics().names().get(); + Map> kafkaFutureMap = _adminClient.describeTopics(topicNames).values(); + KafkaFuture topicDescriptions = kafkaFutureMap.get(_topic); + List partitions = topicDescriptions.get().partitions(); + int partitionNum = partitions.size(); + if (partitionNum < minPartitionNum) { + LOG.info("{} will increase partition of the topic {} in the cluster from {}" + + " to {}.", this.getClass().toString(), _topic, partitionNum, minPartitionNum); + Set blackListedBrokers = + _topicFactory.getBlackListedBrokers(_zkConnect); + List> replicaAssignment = new ArrayList<>(new ArrayList<>()); + Set brokers = new HashSet<>(); + for (Node broker : _adminClient.describeCluster().nodes().get()) { + BrokerMetadata brokerMetadata = new BrokerMetadata( + broker.id(), null + ); + brokers.add(brokerMetadata); + } - try { - scala.collection.Map> existingAssignment = getPartitionAssignment(zkClient, _topic); - int partitionNum = existingAssignment.size(); - - if (partitionNum < minPartitionNum) { - LOG.info("MultiClusterTopicManagementService will increase partition of the topic {} " - + "in cluster {} from {} to {}.", _topic, _zkConnect, partitionNum, minPartitionNum); - Set blackListedBrokers = - _topicFactory.getBlackListedBrokers(_zkConnect); - scala.Option>> replicaAssignment = scala.Option.apply(null); - scala.Option> brokerList = scala.Option.apply(null); - Set brokers = - new HashSet<>(scala.collection.JavaConversions.asJavaCollection(adminZkClient.getBrokerMetadatas(RackAwareMode.Disabled$.MODULE$, brokerList))); - - if (!blackListedBrokers.isEmpty()) { - brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); - } - adminZkClient.addPartitions(_topic, existingAssignment, - scala.collection.JavaConversions.collectionAsScalaIterable(brokers).toSeq(), - minPartitionNum, replicaAssignment, false); + if (!blackListedBrokers.isEmpty()) { + brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); } - } finally { - zkClient.close(); + Map newPartitionsMap = new HashMap<>(); + NewPartitions newPartitions = NewPartitions.increaseTo(minPartitionNum, replicaAssignment); + newPartitionsMap.put(_topic, newPartitions); + _adminClient.createPartitions(newPartitionsMap); } } - private Set getAvailableBrokers(KafkaZkClient zkClient) { - Set brokers = - new HashSet<>(scala.collection.JavaConversions.asJavaCollection(zkClient.getAllBrokersInCluster())); - + private Set getAvailableBrokers() throws ExecutionException, InterruptedException { + Set brokers = new HashSet<>(); + for (Node node : _adminClient.describeCluster().nodes().get()) { + brokers.add(node); + } Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); @@ -296,26 +325,27 @@ private Set getAvailableBrokers(KafkaZkClient zkClient) { } void maybeReassignPartitionAndElectLeader() throws Exception { - KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, - ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); + KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); try { - List partitionInfoList = getPartitionInfo(zkClient, _topic); - Collection brokers = getAvailableBrokers(zkClient); + List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + Collection brokers = getAvailableBrokers(); boolean partitionReassigned = false; - if (partitionInfoList.size() == 0) - throw new IllegalStateException("Topic " + _topic + " does not exist in cluster " + _zkConnect); + if (partitionInfoList.size() == 0) { + throw new IllegalStateException("Topic " + _topic + " does not exist in cluster."); + } int currentReplicationFactor = getReplicationFactor(partitionInfoList); int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); if (_replicationFactor < currentReplicationFactor) - LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster {}", - _replicationFactor, currentReplicationFactor, _topic, _zkConnect); + LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", + _replicationFactor, currentReplicationFactor, _topic); if (expectedReplicationFactor > currentReplicationFactor && !zkClient.reassignPartitionsInProgress()) { - LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster {}" - + "from {} to {}", _topic, _zkConnect, currentReplicationFactor, expectedReplicationFactor); + LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" + + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); partitionReassigned = true; } @@ -330,14 +360,13 @@ void maybeReassignPartitionAndElectLeader() throws Exception { if (!currentProperties.equals(expectedProperties)) { LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " - + "in cluster {} from {} to {}.", _topic, _zkConnect, currentProperties, expectedProperties); + + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); } if (partitionInfoList.size() >= brokers.size() && - someBrokerNotPreferredLeader(partitionInfoList, brokers) && - !zkClient.reassignPartitionsInProgress()) { - LOG.info("MultiClusterTopicManagementService will reassign partitions of the topic {} in cluster {}", _topic, _zkConnect); + someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient.reassignPartitionsInProgress()) { + LOG.info("{} will reassign partitions of the topic {} in cluster.", this.getClass().toString(), _topic); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); partitionReassigned = true; } @@ -346,16 +375,16 @@ void maybeReassignPartitionAndElectLeader() throws Exception { someBrokerNotElectedLeader(partitionInfoList, brokers)) { if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { LOG.info( - "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in cluster {}", - _topic, _zkConnect); - triggerPreferredLeaderElection(zkClient, partitionInfoList); + "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " + + "cluster.", _topic + ); + triggerPreferredLeaderElection(partitionInfoList, _topic); _preferredLeaderElectionRequested = false; } else { _preferredLeaderElectionRequested = true; } } } finally { - zkClient.close(); } } @@ -364,35 +393,36 @@ void maybeElectLeader() throws Exception { return; } - KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), ZK_SESSION_TIMEOUT_MS, - ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener"); + KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); try { if (!zkClient.reassignPartitionsInProgress()) { - List partitionInfoList = getPartitionInfo(zkClient, _topic); - LOG.info( - "MultiClusterTopicManagementService will trigger requested preferred leader election for the topic {} in cluster {}", - _topic, _zkConnect); - triggerPreferredLeaderElection(zkClient, partitionInfoList); + List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + LOG.info("MultiClusterTopicManagementService will trigger requested preferred leader election for the" + + " topic {} in cluster.", _topic); + triggerPreferredLeaderElection(partitionInfoList, _topic); _preferredLeaderElectionRequested = false; } } finally { - zkClient.close(); } } - private static void triggerPreferredLeaderElection(KafkaZkClient zkClient, List partitionInfoList) { - scala.collection.mutable.HashSet scalaPartitionInfoSet = new scala.collection.mutable.HashSet<>(); - for (PartitionInfo javaPartitionInfo : partitionInfoList) { - scalaPartitionInfoSet.add(new TopicPartition(javaPartitionInfo.topic(), javaPartitionInfo.partition())); + private void triggerPreferredLeaderElection(List partitionInfoList, String partitionTopic) + throws ExecutionException, InterruptedException { + Collection partitions = new HashSet<>(); + for (TopicPartitionInfo javaPartitionInfo : partitionInfoList) { + partitions.add(new TopicPartition(partitionTopic, javaPartitionInfo.partition())); } - PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkClient, scalaPartitionInfoSet); + ElectPreferredLeadersResult electPreferredLeadersResult = _adminClient.electPreferredLeaders(partitions); + + LOG.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electPreferredLeadersResult.all().get()); } - private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { + private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { scala.collection.mutable.ArrayBuffer brokersMetadata = new scala.collection.mutable.ArrayBuffer<>(brokers.size()); - for (Broker broker : brokers) { - brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), broker.rack())); + for (Node broker : brokers) { + brokersMetadata.$plus$eq(new BrokerMetadata(broker.id(), Option$.MODULE$.apply(broker.rack()))); } scala.collection.Map> assignedReplicas = AdminUtils.assignReplicasToBrokers(brokersMetadata, partitionCount, replicationFactor, 0, 0); @@ -416,63 +446,35 @@ private static void reassignPartitions(KafkaZkClient zkClient, Collection> getPartitionAssignment(KafkaZkClient zkClient, String topic) { - scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); - return zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); - } - - private static List getPartitionInfo(KafkaZkClient zkClient, String topic) { - scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); - scala.collection.Map> partitionAssignments = - zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); - List partitionInfoList = new ArrayList<>(); - scala.collection.Iterator>> it = partitionAssignments.iterator(); - while (it.hasNext()) { - scala.Tuple2> scalaTuple = it.next(); - Integer partition = (Integer) scalaTuple._1(); - scala.Option leaderOption = zkClient.getLeaderForPartition(new TopicPartition(topic, partition)); - Node leader = leaderOption.isEmpty() ? null : new Node((Integer) leaderOption.get(), "", -1); - Node[] replicas = new Node[scalaTuple._2().size()]; - for (int i = 0; i < replicas.length; i++) { - Integer brokerId = (Integer) scalaTuple._2().apply(i); - replicas[i] = new Node(brokerId, "", -1); - } - partitionInfoList.add(new PartitionInfo(topic, partition, leader, replicas, null)); - } - - return partitionInfoList; - } - - static int getReplicationFactor(List partitionInfoList) { + static int getReplicationFactor(List partitionInfoList) { if (partitionInfoList.isEmpty()) - throw new RuntimeException("Partition list is empty"); + throw new RuntimeException("Partition list is empty."); - int replicationFactor = partitionInfoList.get(0).replicas().length; - for (PartitionInfo partitionInfo : partitionInfoList) { - if (replicationFactor != partitionInfo.replicas().length) { - String topic = partitionInfoList.get(0).topic(); - LOG.warn("Partitions of the topic " + topic + " have different replication factor"); + int replicationFactor = partitionInfoList.get(0).replicas().size(); + for (TopicPartitionInfo partitionInfo : partitionInfoList) { + if (replicationFactor != partitionInfo.replicas().size()) { + LOG.warn("Partitions of the topic have different replication factor."); return -1; } } return replicationFactor; } - static boolean someBrokerNotPreferredLeader(List partitionInfoList, Collection brokers) { + static boolean someBrokerNotPreferredLeader(List partitionInfoList, Collection brokers) { Set brokersNotPreferredLeader = new HashSet<>(brokers.size()); - for (Broker broker: brokers) + for (Node broker: brokers) brokersNotPreferredLeader.add(broker.id()); - for (PartitionInfo partitionInfo : partitionInfoList) - brokersNotPreferredLeader.remove(partitionInfo.replicas()[0].id()); + for (TopicPartitionInfo partitionInfo : partitionInfoList) + brokersNotPreferredLeader.remove(partitionInfo.replicas().get(0).id()); return !brokersNotPreferredLeader.isEmpty(); } - static boolean someBrokerNotElectedLeader(List partitionInfoList, Collection brokers) { + static boolean someBrokerNotElectedLeader(List partitionInfoList, Collection brokers) { Set brokersNotElectedLeader = new HashSet<>(brokers.size()); - for (Broker broker: brokers) + for (Node broker: brokers) brokersNotElectedLeader.add(broker.id()); - for (PartitionInfo partitionInfo : partitionInfoList) { + for (TopicPartitionInfo partitionInfo : partitionInfoList) { if (partitionInfo.leader() != null) brokersNotElectedLeader.remove(partitionInfo.leader().id()); } @@ -510,7 +512,5 @@ private static String formatAsReassignmentJson(String topic, scala.collection.Ma bldr.append("]}"); return bldr.toString(); } - } } - diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 573f185c..056de4c9 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -16,12 +16,14 @@ import com.linkedin.kmf.producer.NewProducer; import com.linkedin.kmf.services.configs.ProduceServiceConfig; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; @@ -29,10 +31,14 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; @@ -77,15 +83,14 @@ public class ProduceService implements Service { private final Map _producerPropsOverride; private final String _producerClassName; private final int _threadsNum; - private final String _zkConnect; private final boolean _treatZeroThroughputAsUnavailable; private final int _latencyPercentileMaxMs; private final int _latencyPercentileGranularityMs; + private final AdminClient _adminClient; public ProduceService(Map props, String name) throws Exception { _name = name; ProduceServiceConfig config = new ProduceServiceConfig(props); - _zkConnect = config.getString(ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG); _brokerList = config.getString(ProduceServiceConfig.BOOTSTRAP_SERVERS_CONFIG); String producerClass = config.getString(ProduceServiceConfig.PRODUCER_CLASS_CONFIG); _latencyPercentileMaxMs = config.getInt(ProduceServiceConfig.LATENCY_PERCENTILE_MAX_MS_CONFIG); @@ -109,6 +114,7 @@ public ProduceService(Map props, String name) throws Exception { throw new ConfigException("Override must not contain " + property + " config."); } } + _adminClient = AdminClient.create(props); if (producerClass.equals(NewProducer.class.getCanonicalName()) || producerClass.equals(NewProducer.class.getSimpleName())) { _producerClassName = NewProducer.class.getCanonicalName(); @@ -130,9 +136,7 @@ public ProduceService(Map props, String name) throws Exception { _sensors = new ProduceMetrics(metrics, tags); } - private void initializeProducer() throws Exception { - Properties producerProps = new Properties(); // Assign default config. This has the lowest priority. producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); @@ -155,10 +159,16 @@ private void initializeProducer() throws Exception { @Override public synchronized void start() { if (_running.compareAndSet(false, true)) { - int partitionNum = Utils.getPartitionNumForTopic(_zkConnect, _topic); - initializeStateForPartitions(partitionNum); - _handleNewPartitionsExecutor.scheduleWithFixedDelay(new NewPartitionHandler(), 1000, 30000, TimeUnit.MILLISECONDS); - LOG.info("{}/ProduceService started", _name); + try { + KafkaFuture> topicDescriptionsFuture = _adminClient.describeTopics(Collections.singleton(_topic)).all(); + Map topicDescriptions = topicDescriptionsFuture.get(); + int partitionNum = topicDescriptions.get(_topic).partitions().size(); + initializeStateForPartitions(partitionNum); + _handleNewPartitionsExecutor.scheduleWithFixedDelay(new NewPartitionHandler(), 1, 30, TimeUnit.SECONDS); + LOG.info("{}/ProduceService started", _name); + } catch (InterruptedException | UnknownTopicOrPartitionException | ExecutionException e) { + LOG.error("Exception occurred while starting produce service: ", e); + } } } @@ -166,7 +176,7 @@ private void initializeStateForPartitions(int partitionNum) { Map keyMapping = generateKeyMappings(partitionNum); for (int partition = 0; partition < partitionNum; partition++) { String key = keyMapping.get(partition); - //This is what preserves sequence numbers across restarts + /* This is what preserves sequence numbers across restarts */ if (!_nextIndexPerPartition.containsKey(partition)) { _nextIndexPerPartition.put(partition, new AtomicLong(0)); _sensors.addPartitionSensors(partition); @@ -198,7 +208,7 @@ public synchronized void stop() { _produceExecutor.shutdown(); _handleNewPartitionsExecutor.shutdown(); _producer.close(); - LOG.info("{}/ProduceService stopped", _name); + LOG.info("{}/ProduceService stopped.", _name); } } @@ -208,9 +218,9 @@ public void awaitShutdown() { _produceExecutor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); _handleNewPartitionsExecutor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.info("Thread interrupted when waiting for {}/ProduceService to shutdown", _name); + LOG.info("Thread interrupted when waiting for {}/ProduceService to shutdown.", _name); } - LOG.info("{}/ProduceService shutdown completed", _name); + LOG.info("{}/ProduceService shutdown completed.", _name); } @Override @@ -354,35 +364,40 @@ public void run() { * sensors are added for the new partitions. */ private class NewPartitionHandler implements Runnable { - public void run() { LOG.debug("{}/ProduceService check partition number for topic {}.", _name, _topic); - - int currentPartitionNum = Utils.getPartitionNumForTopic(_zkConnect, _topic); - if (currentPartitionNum <= 0) { - LOG.info("{}/ProduceService topic {} does not exist.", _name, _topic); - return; - } else if (currentPartitionNum == _partitionNum.get()) { - return; - } - LOG.info("{}/ProduceService detected new partitions of topic {}", _name, _topic); - //TODO: Should the ProduceService exit if we can't restart the producer runnables? - _produceExecutor.shutdown(); try { - _produceExecutor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); + int currentPartitionNum = + _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions().size(); + if (currentPartitionNum <= 0) { + LOG.info("{}/ProduceService topic {} does not exist.", _name, _topic); + return; + } else if (currentPartitionNum == _partitionNum.get()) { + return; + } + LOG.info("{}/ProduceService detected new partitions of topic {}", _name, _topic); + //TODO: Should the ProduceService exit if we can't restart the producer runnables? + _produceExecutor.shutdown(); + try { + _produceExecutor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + _producer.close(); + try { + initializeProducer(); + } catch (Exception e) { + LOG.error("Failed to restart producer.", e); + throw new IllegalStateException(e); + } + _produceExecutor = Executors.newScheduledThreadPool(_threadsNum); + initializeStateForPartitions(currentPartitionNum); + LOG.info("New partitions added to monitoring."); } catch (InterruptedException e) { - throw new IllegalStateException(e); - } - _producer.close(); - try { - initializeProducer(); - } catch (Exception e) { - LOG.error("Failed to restart producer.", e); - throw new IllegalStateException(e); + LOG.error("InterruptedException occurred {}.", e); + } catch (ExecutionException e) { + LOG.error("ExecutionException occurred {}.", e); } - _produceExecutor = Executors.newScheduledThreadPool(_threadsNum); - initializeStateForPartitions(currentPartitionNum); - LOG.info("New partitions added to monitoring."); } } diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java index f4d4d6fd..9351258b 100644 --- a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -3,8 +3,13 @@ */ package com.linkedin.kmf.services; -import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; - +import com.codahale.metrics.MetricRegistry; +import com.linkedin.kmf.common.MbeanAttributeValue; +import com.linkedin.kmf.services.configs.SignalFxMetricsReporterServiceConfig; +import com.signalfx.codahale.metrics.SettableDoubleGauge; +import com.signalfx.codahale.reporter.MetricMetadata; +import com.signalfx.codahale.reporter.SignalFxReporter; +import com.signalfx.endpoint.SignalFxEndpoint; import java.net.URL; import java.util.HashMap; import java.util.List; @@ -12,19 +17,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.codahale.metrics.MetricRegistry; -import com.linkedin.kmf.common.MbeanAttributeValue; -import com.linkedin.kmf.services.configs.SignalFxMetricsReporterServiceConfig; -import com.signalfx.codahale.metrics.SettableDoubleGauge; -import com.signalfx.codahale.reporter.MetricMetadata; -import com.signalfx.codahale.reporter.SignalFxReporter; -import com.signalfx.endpoint.SignalFxEndpoint; - public class SignalFxMetricsReporterService implements Service { private static final Logger LOG = LoggerFactory.getLogger(SignalFxMetricsReporterService.class); @@ -129,7 +125,7 @@ private void captureMetrics() { String mbeanExpr = metricName.substring(0, index); String attributeExpr = metricName.substring(index + 1); - List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + List attributeValues = com.linkedin.kmf.common.Utils.getMBeanAttributeValues(mbeanExpr, attributeExpr); for (final MbeanAttributeValue attributeValue : attributeValues) { String metric = attributeValue.toString(); diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java index a0b2bad7..0534b396 100644 --- a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java @@ -14,17 +14,15 @@ import com.linkedin.kmf.services.configs.StatsdMetricsReporterServiceConfig; import com.timgroup.statsd.NonBlockingStatsDClient; import com.timgroup.statsd.StatsDClient; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.List; import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static com.linkedin.kmf.common.Utils.getMBeanAttributeValues; public class StatsdMetricsReporterService implements Service { private static final Logger LOG = LoggerFactory.getLogger(StatsdMetricsReporterService.class); @@ -97,7 +95,7 @@ private void reportMetrics() { for (String metricName: _metricNames) { String mbeanExpr = metricName.substring(0, metricName.lastIndexOf(":")); String attributeExpr = metricName.substring(metricName.lastIndexOf(":") + 1); - List attributeValues = getMBeanAttributeValues(mbeanExpr, attributeExpr); + List attributeValues = com.linkedin.kmf.common.Utils.getMBeanAttributeValues(mbeanExpr, attributeExpr); for (MbeanAttributeValue attributeValue: attributeValues) { final String statsdMetricName = generateStatsdMetricName(attributeValue.mbean(), attributeValue.attribute()); diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index 26333b2b..cf8f1791 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -14,6 +14,7 @@ import com.linkedin.kmf.services.configs.TopicManagementServiceConfig; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.CompletableFuture; /** @@ -62,6 +63,11 @@ public synchronized void start() { _multiClusterTopicManagementService.start(); } + public CompletableFuture topicManagementReady() { + return _multiClusterTopicManagementService.topicManagementReady(); + } + + @Override public synchronized void stop() { _multiClusterTopicManagementService.stop(); diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index bd1ff8d1..f6f03c6b 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -10,12 +10,13 @@ package com.linkedin.kmf.tests; import com.linkedin.kmf.apps.SingleClusterMonitor; -import com.linkedin.kmf.services.TopicManagementService; import com.linkedin.kmf.services.ConsumeService; import com.linkedin.kmf.services.ProduceService; +import com.linkedin.kmf.services.TopicManagementService; +import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; /* @@ -50,9 +51,15 @@ public BasicEndToEndTest(Map props, String name) throws Exceptio @Override public void start() { _topicManagementService.start(); - _produceService.start(); - _consumeService.start(); - LOG.info(_name + "/BasicEndToEndTest started"); + CompletableFuture completableFuture = _topicManagementService.topicManagementReady(); + completableFuture.thenRun(() -> { + try { + _produceService.start(); + _consumeService.start(); + } finally { + LOG.info("{} /BasicEndToEndTest started.", _name); + } + }); } @Override @@ -60,7 +67,7 @@ public void stop() { _topicManagementService.stop(); _produceService.stop(); _consumeService.stop(); - LOG.info(_name + "/BasicEndToEndTest stopped"); + LOG.info("{} /BasicEndToEndTest stopped.", _name); } @Override diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index f1232d51..daa9e5ab 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -14,9 +14,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; - import java.util.concurrent.atomic.AtomicReference; import org.testng.annotations.Test; @@ -28,25 +25,25 @@ public class KafkaMonitorTest { public void lifecycleTest() throws Exception { KafkaMonitor kafkaMonitor = kafkaMonitor(); - // Nothing should be started - assertEquals(FakeService.startCount.get(), 0); - assertEquals(FakeService.stopCount.get(), 0); + /* Nothing should be started */ + org.testng.Assert.assertEquals(FakeService.startCount.get(), 0); + org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); - // Should accept but ignore start because start has not been called + /* Should accept but ignore start because start has not been called */ kafkaMonitor.stop(); - assertEquals(FakeService.stopCount.get(), 0); + org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); - // Should start + /* Should start */ kafkaMonitor.start(); - assertEquals(FakeService.startCount.get(), 1); + org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); - // Should allow start to be called more than once + /* Should allow start to be called more than once */ kafkaMonitor.stop(); kafkaMonitor.stop(); - assertEquals(FakeService.startCount.get(), 1); - assertEquals(FakeService.stopCount.get(), 1); + org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); + org.testng.Assert.assertEquals(FakeService.stopCount.get(), 1); - // Should be allowed to shutdown more than once. + /* Should be allowed to shutdown more than once. */ kafkaMonitor.awaitShutdown(); kafkaMonitor.awaitShutdown(); } @@ -72,8 +69,8 @@ public void run() { Thread.sleep(100); kafkaMonitor.stop(); t.join(500); - assertFalse(t.isAlive()); - assertEquals(error.get(), null); + org.testng.Assert.assertFalse(t.isAlive()); + org.testng.Assert.assertEquals(error.get(), null); } private KafkaMonitor kafkaMonitor() throws Exception { diff --git a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java index 5d08e3c4..53ae7807 100644 --- a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java @@ -9,25 +9,24 @@ */ package com.linkedin.kmf.services; +import com.linkedin.kmf.services.MultiClusterTopicManagementService.TopicManagementHelper; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import kafka.cluster.Broker; import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.TopicPartitionInfo; import org.testng.Assert; import org.testng.annotations.Test; -import com.linkedin.kmf.services.MultiClusterTopicManagementService.TopicManagementHelper; @Test public class TopicManagementServiceTest { private static final String TOPIC = "kmf-unit-test-topic"; - private List brokers(int brokerCount) { - List brokers = new ArrayList<>(); + private List brokers(int brokerCount) { + List brokers = new ArrayList<>(); for (int i = 0; i < brokerCount; i++) { - brokers.add(new Broker(i, "", -1, null, SecurityProtocol.PLAINTEXT)); + brokers.add(new Node(i, "", -1)); } return brokers; } @@ -42,12 +41,12 @@ private Node[] nodes(int brokerCount) { @Test public void noDetection() { - List partitions = new ArrayList<>(); + List partitions = new ArrayList<>(); Node[] node = nodes(2); - partitions.add(new PartitionInfo(TOPIC, 0, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 1, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 2, node[1], new Node[] {node[1], node[0]}, null)); - partitions.add(new PartitionInfo(TOPIC, 3, node[1], new Node[] {node[1], node[0]}, null)); + partitions.add(new TopicPartitionInfo(0, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(1, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(2, node[1], new ArrayList<>(Arrays.asList(node[1], node[0])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(3, node[1], new ArrayList<>(Arrays.asList(node[1], node[0])), new ArrayList<>())); Assert.assertFalse(TopicManagementHelper.someBrokerNotPreferredLeader(partitions, brokers(2))); Assert.assertFalse(TopicManagementHelper.someBrokerNotElectedLeader(partitions, brokers(2))); @@ -55,12 +54,11 @@ public void noDetection() { @Test public void detectLowTotalNumberOfPartitions() { - List partitions = new ArrayList<>(); + List partitions = new ArrayList<>(); Node[] node = nodes(3); - partitions.add(new PartitionInfo(TOPIC, 0, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 1, node[1], new Node[] {node[1], node[0]}, null)); - partitions.add(new PartitionInfo(TOPIC, 2, node[2], new Node[] {node[2], node[0]}, null)); - + partitions.add(new TopicPartitionInfo(0, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(1, node[1], new ArrayList<>(Arrays.asList(node[1], node[0])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(2, node[2], new ArrayList<>(Arrays.asList(node[2], node[0])), new ArrayList<>())); Assert.assertFalse(TopicManagementHelper.someBrokerNotPreferredLeader(partitions, brokers(3))); Assert.assertFalse(TopicManagementHelper.someBrokerNotElectedLeader(partitions, brokers(3))); Assert.assertEquals(TopicManagementHelper.getReplicationFactor(partitions), 2); @@ -69,13 +67,13 @@ public void detectLowTotalNumberOfPartitions() { @Test public void detectBrokerWithoutLeader() { - List partitions = new ArrayList<>(); + List partitions = new ArrayList<>(); Node[] node = nodes(3); - partitions.add(new PartitionInfo(TOPIC, 0, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 1, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 2, node[1], new Node[] {node[1], node[0]}, null)); - partitions.add(new PartitionInfo(TOPIC, 3, node[1], new Node[] {node[2], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 4, node[1], new Node[] {node[2], node[0]}, null)); + partitions.add(new TopicPartitionInfo(0, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(1, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(2, node[1], new ArrayList<>(Arrays.asList(node[1], node[0])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(3, node[1], new ArrayList<>(Arrays.asList(node[2], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(4, node[1], new ArrayList<>(Arrays.asList(node[2], node[0])), new ArrayList<>())); Assert.assertFalse(TopicManagementHelper.someBrokerNotPreferredLeader(partitions, brokers(3))); Assert.assertTrue(TopicManagementHelper.someBrokerNotElectedLeader(partitions, brokers(3))); @@ -83,13 +81,13 @@ public void detectBrokerWithoutLeader() { @Test public void detectBrokerWithoutPreferredLeader() { - List partitions = new ArrayList<>(); + List partitions = new ArrayList<>(); Node[] node = nodes(3); - partitions.add(new PartitionInfo(TOPIC, 0, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 1, node[0], new Node[] {node[0], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 2, node[1], new Node[] {node[0], node[0]}, null)); - partitions.add(new PartitionInfo(TOPIC, 3, node[1], new Node[] {node[2], node[1]}, null)); - partitions.add(new PartitionInfo(TOPIC, 4, node[1], new Node[] {node[2], node[0]}, null)); + partitions.add(new TopicPartitionInfo(0, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(1, node[0], new ArrayList<>(Arrays.asList(node[0], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(2, node[1], new ArrayList<>(Arrays.asList(node[0], node[0])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(3, node[1], new ArrayList<>(Arrays.asList(node[2], node[1])), new ArrayList<>())); + partitions.add(new TopicPartitionInfo(4, node[1], new ArrayList<>(Arrays.asList(node[2], node[0])), new ArrayList<>())); Assert.assertTrue(TopicManagementHelper.someBrokerNotPreferredLeader(partitions, brokers(3))); Assert.assertTrue(TopicManagementHelper.someBrokerNotElectedLeader(partitions, brokers(3))); From a5a37b1c90277e6e6bc216e76fbd6eef789eaea7 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 Nov 2019 12:40:57 -0800 Subject: [PATCH 048/192] Update CONTRIBUTING.md (#168) Reporting Issues on CONTRIBUTION Markdown file. --- CONTRIBUTING.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 771c1929..7d1d7f31 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -19,3 +19,9 @@ Tips for Getting Your Pull Request Accepted 1. Make sure all new features are tested and the tests pass. 2. Bug fixes must include a test case demonstrating the error that it fixes. + +Reporting Issues +=============== +Please use the [link](https://github.com/linkedin/kafka-monitor/issues/new) for reporting any issues. + + From cfab782adccc268a30361efcc362eb1f03f92ae0 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 Nov 2019 12:55:31 -0800 Subject: [PATCH 049/192] Create SECURITY.md (#167) Create SECURITY.md --- SECURITY.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 SECURITY.md diff --git a/SECURITY.md b/SECURITY.md new file mode 100644 index 00000000..f645bac2 --- /dev/null +++ b/SECURITY.md @@ -0,0 +1,17 @@ +# Security Policy + +## Supported Versions + + + +| Version | Supported | +| ------- | ------------------ | +| 1.1.x | :white_check_mark: | + + + +## Reporting a Vulnerability + +Use this section to tell people how to report a vulnerability. + +Please report a vulnerability on issues at https://github.com/linkedin/kafka-monitor/issues/new. From 6857a7882e494993b1a6a6911174d34f7d1132a0 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 Nov 2019 13:52:12 -0800 Subject: [PATCH 050/192] Update .gitignore (#170) Update GIT IGNORE --- .gitignore | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.gitignore b/.gitignore index b8ae435e..848e31f0 100644 --- a/.gitignore +++ b/.gitignore @@ -2,3 +2,9 @@ .DS_Store build/ logs/ +.classpath +.idea/ +.project +.settings/ + + From 9d8dd95923f131e9b6660fbf398d2a8285c8ad9f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 4 Dec 2019 11:05:40 -0800 Subject: [PATCH 051/192] Clarify usage of kafka monitor at LinkedIn (#172) * Update README.md --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index f72294c2..cef3c128 100644 --- a/README.md +++ b/README.md @@ -19,6 +19,8 @@ monitor topic. This allows Kafka Monitor to detect performance issue on every broker without requiring users to manually manage the partition assignment of the monitor topic. +Kafka Monitor is used in conjunction with different middle-layer services such as li-apache-kafka-clients in order to monitor single clusters, pipeline desination clusters, and other types of clusters as done in Linkedin engineering for real-time cluster healthchecks. + ## Getting Started ### Prerequisites From def98041acfe7f9371e95fd281c571a645a4455c Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 4 Dec 2019 16:48:00 -0800 Subject: [PATCH 052/192] KMF's Contributor Covenant Code of Conduct (#169) Initial revision of CoC. --- CODE_OF_CONDUCT.md | 80 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 CODE_OF_CONDUCT.md diff --git a/CODE_OF_CONDUCT.md b/CODE_OF_CONDUCT.md new file mode 100644 index 00000000..118ac0fd --- /dev/null +++ b/CODE_OF_CONDUCT.md @@ -0,0 +1,80 @@ +# Contributor Covenant Code of Conduct + +## Our Pledge + +In the interest of fostering an open and welcoming environment, we as +contributors and maintainers pledge to making participation in our project and +our community a harassment-free experience for everyone, regardless of age, body +size, disability, ethnicity, sex characteristics, gender identity and expression, +level of experience, education, socio-economic status, nationality, personal +appearance, race, religion, or sexual identity and orientation. + +## Our Standards + +Examples of behavior that contributes to creating a positive environment +include: + +* Using welcoming and inclusive language +* Being respectful of differing viewpoints and experiences +* Gracefully accepting constructive criticism +* Focusing on what is best for the community +* Showing empathy towards other community members + +Examples of unacceptable behavior by participants include: + +* The use of sexualized language or imagery and unwelcome sexual attention or + advances +* Trolling, insulting/derogatory comments, and personal or political attacks +* Public or private harassment +* Publishing others' private information, such as a physical or electronic + address, without explicit permission +* Other conduct which could reasonably be considered inappropriate in a + professional setting + +## Our Responsibilities + +Project maintainers are responsible for clarifying the standards of acceptable +behavior and are expected to take appropriate and fair corrective action in +response to any instances of unacceptable behavior. + +Project maintainers have the right and responsibility to remove, edit, or +reject comments, commits, code, wiki edits, issues, and other contributions +that are not aligned to this Code of Conduct, or to ban temporarily or +permanently any contributor for other behaviors that they deem inappropriate, +threatening, offensive, or harmful. + +## Scope + +This Code of Conduct applies both within project spaces and in public spaces +when an individual is representing the project or its community. Examples of +representing a project or community include using an official project e-mail +address, posting via an official social media account, or acting as an appointed +representative at an online or offline event. Representation of a project may be +further defined and clarified by project maintainers. + +## Enforcement + +Instances of abusive, harassing, or otherwise unacceptable behavior may be +reported by contacting the project team at andchoi@linkedin.com. All +complaints will be reviewed and investigated and will result in a response that +is deemed necessary and appropriate to the circumstances. The project team is +obligated to maintain confidentiality with regard to the reporter of an incident. +Further details of specific enforcement policies may be posted separately. + +Project maintainers who do not follow or enforce the Code of Conduct in good +faith may face temporary or permanent repercussions as determined by other +members of the project's leadership. + +## Attribution + +This Code of Conduct is adapted from the [Contributor Covenant][homepage], version 1.4, +available at https://www.contributor-covenant.org/version/1/4/code-of-conduct.html + +[homepage]: https://www.contributor-covenant.org + +## FAQ + +For LinkedIn Code of Conduct (OSS Code of Conduct) issues or inquiries, Global Compliance & Integrity inbox, please email integrity@linkedin.com. + + + From 447a18a69b4bf4c05b7b13a7165aaea8aa3e0c50 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sat, 7 Dec 2019 12:24:49 -0800 Subject: [PATCH 053/192] no static imports (#174) --- .../kmf/apps/SingleClusterMonitor.java | 34 +++++++++---------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 4eb4c5cc..510587a1 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -32,8 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static net.sourceforge.argparse4j.impl.Arguments.store; - /* * The SingleClusterMonitor app is intended to monitor the performance and availability of a given Kafka cluster. It creates * one producer and one consumer with the given configuration, produces messages with increasing integer in the @@ -97,7 +95,7 @@ private static ArgumentParser argParser() { .description(""); parser.addArgument("--topic") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("TOPIC") @@ -105,14 +103,14 @@ private static ArgumentParser argParser() { .help("Produce messages to this topic and consume message from this topic"); parser.addArgument("--producer-id") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .dest("producerId") .help("The producerId will be used by producer client and encoded in the messages to the topic"); parser.addArgument("--broker-list") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(true) .type(String.class) .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") @@ -120,7 +118,7 @@ private static ArgumentParser argParser() { .help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); parser.addArgument("--zookeeper") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(true) .type(String.class) .metavar("HOST:PORT") @@ -128,7 +126,7 @@ private static ArgumentParser argParser() { .help("The connection string for the zookeeper connection in the form host:port"); parser.addArgument("--record-size") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("RECORD_SIZE") @@ -136,7 +134,7 @@ private static ArgumentParser argParser() { .help("The size of each record."); parser.addArgument("--producer-class") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("PRODUCER_CLASS_NAME") @@ -144,7 +142,7 @@ private static ArgumentParser argParser() { .help("Specify the class of producer. Available choices include newProducer or class name"); parser.addArgument("--consumer-class") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("CONSUMER_CLASS_NAME") @@ -152,7 +150,7 @@ private static ArgumentParser argParser() { .help("Specify the class of consumer. Available choices include oldConsumer, newConsumer, or class name"); parser.addArgument("--producer.config") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("PRODUCER_CONFIG") @@ -160,7 +158,7 @@ private static ArgumentParser argParser() { .help("Producer config properties file."); parser.addArgument("--consumer.config") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("CONSUMER_CONFIG") @@ -168,7 +166,7 @@ private static ArgumentParser argParser() { .help("Consumer config properties file."); parser.addArgument("--report-interval-sec") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("REPORT_INTERVAL_SEC") @@ -176,7 +174,7 @@ private static ArgumentParser argParser() { .help("Interval in sec with which to export stats"); parser.addArgument("--record-delay-ms") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("RECORD_DELAY_MS") @@ -184,7 +182,7 @@ private static ArgumentParser argParser() { .help("The delay in ms before sending next record to the same partition"); parser.addArgument("--latency-percentile-max-ms") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("LATENCY_PERCENTILE_MAX_MS") @@ -193,7 +191,7 @@ private static ArgumentParser argParser() { "The percentile will be reported as Double.POSITIVE_INFINITY if its value exceeds the max value."); parser.addArgument("--latency-percentile-granularity-ms") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(String.class) .metavar("LATENCY_PERCENTILE_GRANULARITY_MS") @@ -201,7 +199,7 @@ private static ArgumentParser argParser() { .help("The granularity in ms of latency percentile metric. This is the width of the bucket used in percentile calculation."); parser.addArgument("--topic-creation-enabled") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(Boolean.class) .metavar("AUTO_TOPIC_CREATION_ENABLED") @@ -209,7 +207,7 @@ private static ArgumentParser argParser() { .help(TopicManagementServiceConfig.TOPIC_CREATION_ENABLED_DOC); parser.addArgument("--replication-factor") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(Integer.class) .metavar("REPLICATION_FACTOR") @@ -217,7 +215,7 @@ private static ArgumentParser argParser() { .help(TopicManagementServiceConfig.TOPIC_REPLICATION_FACTOR_DOC); parser.addArgument("--topic-rebalance-interval-ms") - .action(store()) + .action(net.sourceforge.argparse4j.impl.Arguments.store()) .required(false) .type(Integer.class) .metavar("REBALANCE_MS") From f9c9004d845108f94fce8999d21c90d19efdc214 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 8 Dec 2019 19:09:25 -0800 Subject: [PATCH 054/192] Removal of deprecate the Old Kafka Monitor Partitioner (#177) KMF: Removal of unused, old KMF Partitioner that is deprecated by NewKMFPartitioner. --- .../kmf/partitioner/NewKMPartitioner.java | 1 + .../kmf/partitioner/OldKMPartitioner.java | 17 ----------------- 2 files changed, 1 insertion(+), 17 deletions(-) delete mode 100644 src/main/java/com/linkedin/kmf/partitioner/OldKMPartitioner.java diff --git a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java index e73a4119..08e0dd0d 100644 --- a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java +++ b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java @@ -21,4 +21,5 @@ public int partition(String key, int partitionNum) { private static int toPositive(int number) { return number & 0x7fffffff; } + } diff --git a/src/main/java/com/linkedin/kmf/partitioner/OldKMPartitioner.java b/src/main/java/com/linkedin/kmf/partitioner/OldKMPartitioner.java deleted file mode 100644 index fa356eff..00000000 --- a/src/main/java/com/linkedin/kmf/partitioner/OldKMPartitioner.java +++ /dev/null @@ -1,17 +0,0 @@ -/** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this - * file except in compliance with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - */ -package com.linkedin.kmf.partitioner; - -public class OldKMPartitioner implements KMPartitioner { - - public int partition(String key, int partitionNum) { - return Math.abs(key.hashCode()) % partitionNum; - } -} From 1b8dae5f054a674f4fbbed897d06309489731b70 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 9 Dec 2019 10:58:23 -0800 Subject: [PATCH 055/192] Executor Fixed rated schedulings Runnable -> lamda conversions (#176) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 👍 KMF Lamda expressions for some of the executor schedulings and add local variable for delay and period seconds.🥇 --- .../java/com/linkedin/kmf/KafkaMonitor.java | 21 +++++++++---------- .../kmf/common/DefaultTopicSchema.java | 8 +++---- .../linkedin/kmf/services/ConsumeService.java | 13 +++++------- .../GraphiteMetricsReporterService.java | 18 +++++++--------- .../services/KafkaMetricsReporterService.java | 13 +++++------- .../MultiClusterTopicManagementService.java | 2 +- .../SignalFxMetricsReporterService.java | 13 +++++------- .../StatsdMetricsReporterService.java | 18 +++++++--------- 8 files changed, 44 insertions(+), 62 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index 9fdcdda0..3c692d32 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -101,17 +101,16 @@ public synchronized void start() { entry.getValue().start(); } - _executor.scheduleAtFixedRate( - new Runnable() { - @Override - public void run() { - try { - checkHealth(); - } catch (Exception e) { - LOG.error("Failed to check health of tests and services", e); - } - } - }, 5, 5, TimeUnit.SECONDS + long initialDelaySecond = 5; + long periodSecond = 5; + + _executor.scheduleAtFixedRate(() -> { + try { + checkHealth(); + } catch (Exception e) { + LOG.error("Failed to check health of tests and services", e); + } + }, initialDelaySecond, periodSecond, TimeUnit.SECONDS ); } diff --git a/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java b/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java index e248807a..a18053be 100644 --- a/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java +++ b/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java @@ -15,17 +15,17 @@ public class DefaultTopicSchema { - public static final Field TOPIC_FIELD = new Field("topic", Schema.create(Schema.Type.STRING), null, null); + static final Field TOPIC_FIELD = new Field("topic", Schema.create(Schema.Type.STRING), null, null); public static final Field TIME_FIELD = new Field("time", Schema.create(Schema.Type.LONG), null, null); public static final Field INDEX_FIELD = new Field("index", Schema.create(Schema.Type.LONG), null, null); - public static final Field PRODUCER_ID_FIELD = new Field("producerId", Schema.create(Schema.Type.STRING), null, null); + static final Field PRODUCER_ID_FIELD = new Field("producerId", Schema.create(Schema.Type.STRING), null, null); - public static final Field CONTENT_FIELD = new Field("content", Schema.create(Schema.Type.STRING), null, null); + static final Field CONTENT_FIELD = new Field("content", Schema.create(Schema.Type.STRING), null, null); - public static final Schema MESSAGE_V0; + static final Schema MESSAGE_V0; static { MESSAGE_V0 = Schema.createRecord("KafkaMonitorSchema", null, "kafka.monitor", false); diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 5b447f54..a3df08b8 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -103,14 +103,11 @@ public ConsumeService(Map props, String name) throws Exception { _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); - _thread = new Thread(new Runnable() { - @Override - public void run() { - try { - consume(); - } catch (Exception e) { - LOG.error(_name + "/ConsumeService failed", e); - } + _thread = new Thread(() -> { + try { + consume(); + } catch (Exception e) { + LOG.error(_name + "/ConsumeService failed", e); } }, _name + " consume-service"); _thread.setDaemon(true); diff --git a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java index bb3f72b9..a9025e1f 100644 --- a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java @@ -51,17 +51,13 @@ public GraphiteMetricsReporterService(Map props, String name) @Override public synchronized void start() { - _executor.scheduleAtFixedRate( - new Runnable() { - @Override - public void run() { - try { - reportMetrics(); - } catch (Exception e) { - LOG.error(_name + "/GraphiteMetricsReporterService failed to report metrics", e); - } - } - }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS + _executor.scheduleAtFixedRate(() -> { + try { + reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/GraphiteMetricsReporterService failed to report metrics", e); + } + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS ); LOG.info("{}/GraphiteMetricsReporterService started", _name); } diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index 1294ead7..930d0bcd 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -64,14 +64,11 @@ public KafkaMetricsReporterService(Map props, String name) throw @Override public synchronized void start() { - _executor.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - try { - reportMetrics(); - } catch (Exception e) { - LOG.error(_name + "/KafkaMetricsReporterService failed to report metrics", e); - } + _executor.scheduleAtFixedRate(() -> { + try { + reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/KafkaMetricsReporterService failed to report metrics", e); } }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); LOG.info("{}/KafkaMetricsReporterService started", _name); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 41cb42a3..0f7370f7 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -271,7 +271,7 @@ void maybeCreateTopic() throws Exception { } } - public AdminClient constructAdminClient(Map props) { + AdminClient constructAdminClient(Map props) { props.putIfAbsent(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, _bootstrapServers); props.putIfAbsent(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, _requestTimeoutMs); return AdminClient.create(props); diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java index 9351258b..389d4b53 100644 --- a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -73,14 +73,11 @@ public SignalFxMetricsReporterService(Map props, String name) th @Override public synchronized void start() { _signalfxReporter.start(_reportIntervalSec, TimeUnit.SECONDS); - _executor.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - try { - captureMetrics(); - } catch (Exception e) { - LOG.error(_name + "/SignalFxMetricsReporterService failed to report metrics", e); - } + _executor.scheduleAtFixedRate(() -> { + try { + captureMetrics(); + } catch (Exception e) { + LOG.error(_name + "/SignalFxMetricsReporterService failed to report metrics", e); } }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); LOG.info("{}/SignalFxMetricsReporterService started", _name); diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java index 0534b396..a9603807 100644 --- a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java @@ -47,17 +47,13 @@ public StatsdMetricsReporterService(Map props, String name) { @Override public synchronized void start() { - _executor.scheduleAtFixedRate( - new Runnable() { - @Override - public void run() { - try { - reportMetrics(); - } catch (Exception e) { - LOG.error(_name + "/StatsdMetricsReporterService failed to report metrics", e); - } - } - }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS + _executor.scheduleAtFixedRate(() -> { + try { + reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/StatsdMetricsReporterService failed to report metrics", e); + } + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS ); LOG.info("{}/StatsdMetricsReporterService started", _name); } From fd12d19323d29015d103cf98b4763356f7428503 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 9 Dec 2019 18:16:46 -0800 Subject: [PATCH 056/192] Migration from ZkClient to AdminClient for KMF Utils Class (#175) adminClient for Utils java class update getPartitionNumForTopic with adminClient remove zkUrl for topic factories readability commnets for the kafkametricsreporterservices. Suppress warnings and some style check amendments Signed-off-by: Andrew Choi --- .../kmf/apps/SingleClusterMonitor.java | 9 +- .../java/com/linkedin/kmf/common/Utils.java | 96 +++++++------- .../linkedin/kmf/consumer/KMBaseConsumer.java | 2 +- .../linkedin/kmf/services/ConsumeService.java | 60 +++++---- .../services/KafkaMetricsReporterService.java | 27 ++-- .../MultiClusterTopicManagementService.java | 125 ++++++++---------- .../kmf/topicfactory/DefaultTopicFactory.java | 8 +- .../kmf/topicfactory/TopicFactory.java | 6 +- 8 files changed, 162 insertions(+), 171 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 510587a1..46a099fa 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -233,7 +233,6 @@ public static void main(String[] args) throws Exception { } Namespace res = parser.parseArgs(args); - Map props = new HashMap<>(); // produce service config props.put(ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG, res.getString("zkConnect")); @@ -279,6 +278,7 @@ public static void main(String[] args) throws Exception { if (res.getString("reportIntervalSec") != null) props.put(DefaultMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG, res.getString("reportIntervalSec")); List metrics = Arrays.asList( + "kmf.services:type=consume-service,name=*:topic-partitions-count", "kmf.services:type=produce-service,name=*:produce-availability-avg", "kmf.services:type=consume-service,name=*:consume-availability-avg", "kmf.services:type=produce-service,name=*:records-produced-total", @@ -289,16 +289,17 @@ public static void main(String[] args) throws Exception { "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", "kmf.services:type=produce-service,name=*:produce-error-rate", - "kmf.services:type=consume-service,name=*:consume-error-rate"); + "kmf.services:type=consume-service,name=*:consume-error-rate" + ); props.put(DefaultMetricsReporterServiceConfig.REPORT_METRICS_CONFIG, metrics); DefaultMetricsReporterService metricsReporterService = new DefaultMetricsReporterService(props, "end-to-end"); metricsReporterService.start(); - JolokiaService jolokiaService = new JolokiaService(new HashMap(), "end-to-end"); + JolokiaService jolokiaService = new JolokiaService(new HashMap<>(), "end-to-end"); jolokiaService.start(); - JettyService jettyService = new JettyService(new HashMap(), "end-to-end"); + JettyService jettyService = new JettyService(new HashMap<>(), "end-to-end"); jettyService.start(); if (!app.isRunning()) { diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index ab8ce2fb..f8815d2b 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -13,113 +13,108 @@ import java.io.IOException; import java.lang.management.ManagementFactory; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Properties; import java.util.Set; - -import kafka.admin.AdminUtils; -import kafka.admin.RackAwareMode; +import java.util.concurrent.ExecutionException; +import javax.management.MBeanAttributeInfo; +import javax.management.MBeanInfo; +import javax.management.MBeanServer; +import javax.management.ObjectName; import kafka.server.KafkaConfig; -import kafka.utils.ZkUtils; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.Encoder; import org.apache.avro.io.JsonEncoder; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.errors.TopicExistsException; -import org.apache.kafka.common.security.JaasUtils; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.collection.Seq; - -import javax.management.MBeanAttributeInfo; -import javax.management.MBeanInfo; -import javax.management.MBeanServer; -import javax.management.ObjectName; - /** - * Kafka monitoring utilities. + * Kafka Monitoring utilities. */ public class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); - public static final int ZK_CONNECTION_TIMEOUT_MS = 30_000; public static final int ZK_SESSION_TIMEOUT_MS = 30_000; /** - * Read number of partitions for the given topic on the specified zookeeper - * @param zkUrl zookeeper connection url - * @param topic topic name - * + * Read number of partitions for the given topic on the specified ZooKeeper + * @param adminClient AdminClient object initialized. + * @param topic topic name. * @return the number of partitions of the given topic + * @throws ExecutionException thrown when describeTopics(topics) get(topic) execution fails. + * @throws InterruptedException thrown when adminClient's describeTopics getTopic is interrupted. */ - public static int getPartitionNumForTopic(String zkUrl, String topic) { - ZkUtils zkUtils = ZkUtils.apply(zkUrl, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); + private static int getPartitionNumForTopic(AdminClient adminClient, String topic) + throws ExecutionException, InterruptedException { try { - Seq topics = scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(topic)); - return zkUtils.getPartitionsForTopics(topics).apply(topic).size(); + return adminClient.describeTopics(Collections.singleton(topic)).values().get(topic).get().partitions().size(); } catch (NoSuchElementException e) { return 0; } finally { - zkUtils.close(); + LOG.info("Finished getPartitionNumForTopic."); } } /** - * Create the topic. This method attempts to create a topic so that all + * Create the topic. This method attempts to create a topic so that all * the brokers in the cluster will have partitionToBrokerRatio partitions. If the topic exists, but has different parameters * then this does nothing to update the parameters. * * TODO: Do we care about rack aware mode? I would think no because we want to spread the topic over all brokers. - * @param zkUrl zookeeper connection url * @param topic topic name * @param replicationFactor the replication factor for the topic * @param partitionToBrokerRatio This is multiplied by the number brokers to compute the number of partitions in the topic. * @param minPartitionNum partition number to be created at least * @param topicConfig additional parameters for the topic for example min.insync.replicas + * @param adminClient AdminClient object initialized. * @return the number of partitions created + * @throws ExecutionException exception thrown then executing the topic creation fails. + * @throws InterruptedException exception that's thrown when interrupt occurs. */ - public static int createTopicIfNotExists(String zkUrl, String topic, int replicationFactor, - double partitionToBrokerRatio, int minPartitionNum, Properties topicConfig) { - ZkUtils zkUtils = ZkUtils.apply(zkUrl, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); + @SuppressWarnings("unchecked") + public static int createTopicIfNotExists(String topic, short replicationFactor, double partitionToBrokerRatio, + int minPartitionNum, Properties topicConfig, AdminClient adminClient) + throws ExecutionException, InterruptedException { try { - if (AdminUtils.topicExists(zkUtils, topic)) { - return getPartitionNumForTopic(zkUrl, topic); + if (adminClient.listTopics().names().get().contains(topic)) { + return getPartitionNumForTopic(adminClient, topic); } - int brokerCount = zkUtils.getAllBrokersInCluster().size(); + int brokerCount = Utils.getBrokerCount(adminClient); int partitionCount = Math.max((int) Math.ceil(brokerCount * partitionToBrokerRatio), minPartitionNum); - try { - AdminUtils.createTopic(zkUtils, topic, partitionCount, replicationFactor, topicConfig, RackAwareMode.Enforced$.MODULE$); + NewTopic newTopic = new NewTopic(topic, partitionCount, replicationFactor); + newTopic.configs((Map) topicConfig); + List topics = new ArrayList(); + topics.add(newTopic); + adminClient.createTopics(topics); } catch (TopicExistsException e) { - // There is a race condition with the consumer. - LOG.debug("Monitoring topic " + topic + " already exists in cluster " + zkUrl, e); - return getPartitionNumForTopic(zkUrl, topic); + /* There is a race condition with the consumer. */ + LOG.debug("Monitoring topic " + topic + " already exists in the cluster.", e); + return getPartitionNumForTopic(adminClient, topic); } - LOG.info("Created monitoring topic " + topic + " in cluster " + zkUrl + " with " + partitionCount + " partitions, min ISR of " + LOG.info("Created monitoring topic " + topic + " in cluster with " + partitionCount + " partitions, min ISR of " + topicConfig.get(KafkaConfig.MinInSyncReplicasProp()) + " and replication factor of " + replicationFactor + "."); return partitionCount; } finally { - zkUtils.close(); + LOG.info("Completed the topic creation if it doesn't exist."); } } /** - * @param zkUrl zookeeper connection url - * @return number of brokers in this cluster + * @return the number of brokers in this cluster */ - public static int getBrokerCount(String zkUrl) { - ZkUtils zkUtils = ZkUtils.apply(zkUrl, ZK_SESSION_TIMEOUT_MS, ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); - try { - return zkUtils.getAllBrokersInCluster().size(); - } finally { - zkUtils.close(); - } + private static int getBrokerCount(AdminClient adminClient) throws ExecutionException, InterruptedException { + return adminClient.describeCluster().nodes().get().size(); } /** @@ -142,7 +137,7 @@ public static String jsonFromFields(String topic, long idx, long timestamp, Stri /** * @param message kafka message in the string format - * @return GenericRecord that is deserialized from kafka message w.r.t. expected schema + * @return GenericRecord that is de-serialized from kafka message w.r.t. expected schema. */ public static GenericRecord genericRecordFromJson(String message) { GenericRecord record = new GenericData.Record(DefaultTopicSchema.MESSAGE_V0); @@ -155,7 +150,7 @@ public static GenericRecord genericRecordFromJson(String message) { return record; } - public static String jsonFromGenericRecord(GenericRecord record) { + private static String jsonFromGenericRecord(GenericRecord record) { ByteArrayOutputStream out = new ByteArrayOutputStream(); GenericDatumWriter writer = new GenericDatumWriter<>(DefaultTopicSchema.MESSAGE_V0); @@ -189,5 +184,4 @@ public static List getMBeanAttributeValues(String mbeanExpr } return values; } - } diff --git a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java index ad3ce1e4..6b3ba13f 100644 --- a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java @@ -21,4 +21,4 @@ public interface KMBaseConsumer { void close(); -} \ No newline at end of file +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index a3df08b8..0475519b 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -16,19 +16,24 @@ import com.linkedin.kmf.consumer.NewConsumer; import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; @@ -47,7 +52,7 @@ public class ConsumeService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ConsumeService.class); private static final String METRIC_GROUP_NAME = "consume-service"; - private static final String[] NONOVERRIDABLE_PROPERTIES = + private static final String[] NON_OVERRIDABLE_PROPERTIES = new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; @@ -59,6 +64,7 @@ public class ConsumeService implements Service { private final int _latencyPercentileGranularityMs; private final AtomicBoolean _running; private final int _latencySlaMs; + private final AdminClient _adminClient; public ConsumeService(Map props, String name) throws Exception { _name = name; @@ -73,8 +79,7 @@ public ConsumeService(Map props, String name) throws Exception { _latencyPercentileMaxMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_MAX_MS_CONFIG); _latencyPercentileGranularityMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG); _running = new AtomicBoolean(false); - - for (String property: NONOVERRIDABLE_PROPERTIES) { + for (String property: NON_OVERRIDABLE_PROPERTIES) { if (consumerPropsOverride.containsKey(property)) { throw new ConfigException("Override must not contain " + property + " config."); } @@ -102,7 +107,6 @@ public ConsumeService(Map props, String name) throws Exception { consumerProps.putAll(consumerPropsOverride); _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); - _thread = new Thread(() -> { try { consume(); @@ -118,7 +122,8 @@ public ConsumeService(Map props, String name) throws Exception { Metrics metrics = new Metrics(metricConfig, reporters, new SystemTime()); Map tags = new HashMap<>(); tags.put("name", _name); - _sensors = new ConsumeMetrics(metrics, tags); + _adminClient = AdminClient.create(props); + _sensors = new ConsumeMetrics(metrics, tags, topic); } private void consume() throws Exception { @@ -216,7 +221,14 @@ private class ConsumeMetrics { private final Sensor _recordsDelay; private final Sensor _recordsDelayed; - public ConsumeMetrics(final Metrics metrics, final Map tags) { + ConsumeMetrics(final Metrics metrics, final Map tags, String topicName) throws ExecutionException, InterruptedException { + int partitionCount = 0; + DescribeTopicsResult describeTopicsResult = _adminClient.describeTopics(Collections.singleton(topicName)); + Map> values = describeTopicsResult.values(); + partitionCount = values.get(topicName).get().partitions().size(); + Sensor topicPartitionCount = metrics.sensor("topic-partitions"); + topicPartitionCount.add(new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), new Total(partitionCount)); + _bytesConsumed = metrics.sensor("bytes-consumed"); _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); @@ -253,27 +265,19 @@ public ConsumeMetrics(final Metrics metrics, final Map tags) { new Percentile(new MetricName("records-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of records from producer to consumer", tags), 99.99))); metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), - new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); - - if (new Double(recordsLostRate).isNaN()) - recordsLostRate = 0; - if (new Double(recordsDelayedRate).isNaN()) - recordsDelayedRate = 0; - - double consumeAvailability = recordsConsumedRate + recordsLostRate > 0 - ? (recordsConsumedRate - recordsDelayedRate) / (recordsConsumedRate + recordsLostRate) : 0; - - return consumeAvailability; - } - } - ); + (config, now) -> { + double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); + + if (new Double(recordsLostRate).isNaN()) + recordsLostRate = 0; + if (new Double(recordsDelayedRate).isNaN()) + recordsDelayedRate = 0; + + return recordsConsumedRate + recordsLostRate > 0 + ? (recordsConsumedRate - recordsDelayedRate) / (recordsConsumedRate + recordsLostRate) : 0; + }); } - } - } \ No newline at end of file diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index 930d0bcd..c861823c 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -21,6 +21,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -28,38 +29,34 @@ import org.slf4j.LoggerFactory; public class KafkaMetricsReporterService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricsReporterService.class); private static final String METRICS_PRODUCER_ID = "kafka-metrics-reporter-id"; - private final String _name; private final List _metricsNames; private final int _reportIntervalSec; private final ScheduledExecutorService _executor; - private KafkaProducer _producer; private final String _brokerList; private final String _topic; - private final ObjectMapper _parser = new ObjectMapper(); - public KafkaMetricsReporterService(Map props, String name) throws Exception { + public KafkaMetricsReporterService(Map props, String name, AdminClient adminClient) throws Exception { _name = name; KafkaMetricsReporterServiceConfig config = new KafkaMetricsReporterServiceConfig(props); _metricsNames = config.getList(KafkaMetricsReporterServiceConfig.REPORT_METRICS_CONFIG); _reportIntervalSec = config.getInt(KafkaMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG); _executor = Executors.newSingleThreadScheduledExecutor(); - _brokerList = config.getString(KafkaMetricsReporterServiceConfig.BOOTSTRAP_SERVERS_CONFIG); initializeProducer(); - _topic = config.getString(KafkaMetricsReporterServiceConfig.TOPIC_CONFIG); - Utils.createTopicIfNotExists(config.getString(KafkaMetricsReporterServiceConfig.ZOOKEEPER_CONNECT_CONFIG), - _topic, - config.getInt(KafkaMetricsReporterServiceConfig.TOPIC_REPLICATION_FACTOR), - 0, - 1, // fixed partition count 1 - new Properties()); + Utils.createTopicIfNotExists( + _topic, + config.getShort(KafkaMetricsReporterServiceConfig.TOPIC_REPLICATION_FACTOR), + 0, // parameter is set to 0 here since no matter the number of nodes, the topic partition number should be set to zero. + 1, // fixed partition count 1 + new Properties(), + adminClient + ); } @Override @@ -96,7 +93,7 @@ public void awaitShutdown() { LOG.info("{}/KafkaMetricsReporterService shutdown completed", _name); } - private void initializeProducer() throws Exception { + private void initializeProducer() { Properties producerProps = new Properties(); producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000"); @@ -125,7 +122,7 @@ private void reportMetrics() { } try { LOG.debug("Kafka Metrics Reporter sending metrics = " + _parser.writerWithDefaultPrettyPrinter().writeValueAsString(metrics)); - _producer.send(new ProducerRecord(_topic, _parser.writeValueAsString(metrics))); + _producer.send(new ProducerRecord<>(_topic, _parser.writeValueAsString(metrics))); } catch (JsonProcessingException e) { LOG.warn("unsupported json format: " + metrics, e); } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 0f7370f7..dc8676d0 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -236,6 +236,7 @@ static class TopicManagementHelper { private List _bootstrapServers; private final AdminClient _adminClient; + @SuppressWarnings("unchecked") TopicManagementHelper(Map props) throws Exception { TopicManagementServiceConfig config = new TopicManagementServiceConfig(props); AdminClientConfig adminClientConfig = new AdminClientConfig(props); @@ -260,6 +261,7 @@ static class TopicManagementHelper { _adminClient = constructAdminClient(props); } + @SuppressWarnings("unchecked") void maybeCreateTopic() throws Exception { if (_topicCreationEnabled) { int brokerCount = _adminClient.describeCluster().nodes().get().size(); @@ -291,8 +293,7 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup if (partitionNum < minPartitionNum) { LOG.info("{} will increase partition of the topic {} in the cluster from {}" + " to {}.", this.getClass().toString(), _topic, partitionNum, minPartitionNum); - Set blackListedBrokers = - _topicFactory.getBlackListedBrokers(_zkConnect); + Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); List> replicaAssignment = new ArrayList<>(new ArrayList<>()); Set brokers = new HashSet<>(); for (Node broker : _adminClient.describeCluster().nodes().get()) { @@ -314,12 +315,8 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup private Set getAvailableBrokers() throws ExecutionException, InterruptedException { Set brokers = new HashSet<>(); - for (Node node : _adminClient.describeCluster().nodes().get()) { - brokers.add(node); - } - Set blackListedBrokers = - _topicFactory.getBlackListedBrokers(_zkConnect); - + brokers.addAll(_adminClient.describeCluster().nodes().get()); + Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); return brokers; } @@ -328,63 +325,60 @@ void maybeReassignPartitionAndElectLeader() throws Exception { KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); - try { - List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); - Collection brokers = getAvailableBrokers(); - boolean partitionReassigned = false; - if (partitionInfoList.size() == 0) { - throw new IllegalStateException("Topic " + _topic + " does not exist in cluster."); - } + List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + Collection brokers = this.getAvailableBrokers(); + boolean partitionReassigned = false; + if (partitionInfoList.size() == 0) { + throw new IllegalStateException("Topic " + _topic + " does not exist in cluster."); + } - int currentReplicationFactor = getReplicationFactor(partitionInfoList); - int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); + int currentReplicationFactor = getReplicationFactor(partitionInfoList); + int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); - if (_replicationFactor < currentReplicationFactor) - LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", - _replicationFactor, currentReplicationFactor, _topic); + if (_replicationFactor < currentReplicationFactor) + LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", + _replicationFactor, currentReplicationFactor, _topic); - if (expectedReplicationFactor > currentReplicationFactor && !zkClient.reassignPartitionsInProgress()) { - LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" - + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); - reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); - partitionReassigned = true; - } + if (expectedReplicationFactor > currentReplicationFactor && !zkClient.reassignPartitionsInProgress()) { + LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" + + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + partitionReassigned = true; + } - // Update the properties of the monitor topic if any config is different from the user-specified config - Properties currentProperties = zkClient.getEntityConfigs(ConfigType.Topic(), _topic); - Properties expectedProperties = new Properties(); - for (Object key: currentProperties.keySet()) - expectedProperties.put(key, currentProperties.get(key)); - for (Object key: _topicProperties.keySet()) - expectedProperties.put(key, _topicProperties.get(key)); - - if (!currentProperties.equals(expectedProperties)) { - LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " - + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); - zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); - } + // Update the properties of the monitor topic if any config is different from the user-specified config + Properties currentProperties = zkClient.getEntityConfigs(ConfigType.Topic(), _topic); + Properties expectedProperties = new Properties(); + for (Object key: currentProperties.keySet()) + expectedProperties.put(key, currentProperties.get(key)); + for (Object key: _topicProperties.keySet()) + expectedProperties.put(key, _topicProperties.get(key)); + + if (!currentProperties.equals(expectedProperties)) { + LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); + zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); + } - if (partitionInfoList.size() >= brokers.size() && - someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient.reassignPartitionsInProgress()) { - LOG.info("{} will reassign partitions of the topic {} in cluster.", this.getClass().toString(), _topic); - reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); - partitionReassigned = true; - } + if (partitionInfoList.size() >= brokers.size() && + someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient.reassignPartitionsInProgress()) { + LOG.info("{} will reassign partitions of the topic {} in cluster.", this.getClass().toString(), _topic); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); + partitionReassigned = true; + } - if (partitionInfoList.size() >= brokers.size() && - someBrokerNotElectedLeader(partitionInfoList, brokers)) { - if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { - LOG.info( - "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " - + "cluster.", _topic - ); - triggerPreferredLeaderElection(partitionInfoList, _topic); - _preferredLeaderElectionRequested = false; - } else { - _preferredLeaderElectionRequested = true; - } + if (partitionInfoList.size() >= brokers.size() && + someBrokerNotElectedLeader(partitionInfoList, brokers)) { + if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { + LOG.info( + "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " + + "cluster.", _topic + ); + triggerPreferredLeaderElection(partitionInfoList, _topic); + _preferredLeaderElectionRequested = false; + } else { + _preferredLeaderElectionRequested = true; } - } finally { } } @@ -396,15 +390,12 @@ void maybeElectLeader() throws Exception { KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); - try { - if (!zkClient.reassignPartitionsInProgress()) { - List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); - LOG.info("MultiClusterTopicManagementService will trigger requested preferred leader election for the" - + " topic {} in cluster.", _topic); - triggerPreferredLeaderElection(partitionInfoList, _topic); - _preferredLeaderElectionRequested = false; - } - } finally { + if (!zkClient.reassignPartitionsInProgress()) { + List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + LOG.info("MultiClusterTopicManagementService will trigger requested preferred leader election for the" + + " topic {} in cluster.", _topic); + triggerPreferredLeaderElection(partitionInfoList, _topic); + _preferredLeaderElectionRequested = false; } } diff --git a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java index 5ca24acf..96803a80 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java @@ -10,11 +10,12 @@ package com.linkedin.kmf.topicfactory; import com.linkedin.kmf.common.Utils; - import java.util.Collections; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; public class DefaultTopicFactory implements TopicFactory { @@ -24,8 +25,9 @@ public DefaultTopicFactory(Map config) { } @Override - public int createTopicIfNotExist(String zkUrl, String topic, int replicationFactor, double partitionToBrokerRatio, Properties topicConfig) { - return Utils.createTopicIfNotExists(zkUrl, topic, replicationFactor, partitionToBrokerRatio, 1, topicConfig); + public int createTopicIfNotExist(String topic, short replicationFactor, double partitionToBrokerRatio, Properties topicConfig, AdminClient adminClient) + throws ExecutionException, InterruptedException { + return Utils.createTopicIfNotExists(topic, replicationFactor, partitionToBrokerRatio, 1, topicConfig, adminClient); } @Override diff --git a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java index 54f1dc30..45b4da7c 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java @@ -11,6 +11,8 @@ import java.util.Properties; import java.util.Set; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; /** @@ -24,7 +26,6 @@ public interface TopicFactory { /** * Creates the specified topic if it does not exist. - * @param zkUrl zookeeper connection url * @param topic topic name * @param replicationFactor the replication factor for the topic * @param partitionToBrokerRatio This is multiplied by the number brokers to compute the number of partitions in the topic. @@ -33,7 +34,8 @@ public interface TopicFactory { * @return The number of partitions for the specified topic. */ - int createTopicIfNotExist(String zkUrl, String topic, int replicationFactor, double partitionToBrokerRatio, Properties topicProperties); + int createTopicIfNotExist(String topic, short replicationFactor, double partitionToBrokerRatio, Properties topicProperties, AdminClient adminClient) + throws ExecutionException, InterruptedException; /** * @param zkUrl zookeeper connection url From 478b724046f576214ecf0fa0f16eb50c678b6163 Mon Sep 17 00:00:00 2001 From: Sean McCauliff Date: Tue, 10 Dec 2019 14:31:59 -0800 Subject: [PATCH 057/192] Automatically generate the configuration documentation in html. (#61) Add topic-management.topic.props config to specify topic properties (#75) Automatically generate the configuration documentation in html. Remove document directory creation. Change cluster monitor class name. Signed-off-by: Andrew Choi --- build.gradle | 22 +++++-- .../common/ConfigDocumentationGenerator.java | 65 +++++++++++++++++++ .../MultiClusterTopicManagementService.java | 4 ++ 3 files changed, 87 insertions(+), 4 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java diff --git a/build.gradle b/build.gradle index e57b22ee..423deac8 100644 --- a/build.gradle +++ b/build.gradle @@ -1,4 +1,6 @@ +def configDocDir = "${buildDir}/configDocs" + buildscript { repositories { jcenter() @@ -7,10 +9,10 @@ buildscript { classpath "com.jfrog.bintray.gradle:gradle-bintray-plugin:1.8.4" } } - apply plugin: 'maven-publish' apply plugin: 'com.jfrog.bintray' + allprojects { apply plugin: 'idea' apply plugin: 'eclipse' @@ -126,6 +128,20 @@ allprojects { configProperties = ["suppressionFile": new File(rootDir, "checkstyle/suppressions.xml")] } + task createConfigDocs( dependsOn : compileJava, type : JavaExec) { + classpath sourceSets.main.runtimeClasspath + main = 'com.linkedin.kmf.common.ConfigDocumentationGenerator' + args = [configDocDir, + 'com.linkedin.kmf.services.configs.ConsumeServiceConfig', + 'com.linkedin.kmf.services.configs.DefaultMetricsReporterServiceConfig', + 'com.linkedin.kmf.services.configs.JettyServiceConfig', + 'com.linkedin.kmf.services.configs.ProduceServiceConfig', + 'com.linkedin.kmf.services.configs.TopicManagementServiceConfig', + 'com.linkedin.kmf.apps.configs.MultiClusterMonitorConfig'] + } + + build.dependsOn createConfigDocs + test.dependsOn('checkstyleMain', 'checkstyleTest') test { @@ -141,9 +157,7 @@ allprojects { bintray { user = System.getenv('BINTRAY_USER') key = System.getenv('BINTRAY_KEY') - publications = ['MyPublication'] - pkg { repo = 'maven' name = 'kafka-monitor' @@ -156,8 +170,8 @@ bintray { publish = true } } - wrapper { gradleVersion = '5.2.1' distributionType = Wrapper.DistributionType.ALL + } diff --git a/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java new file mode 100644 index 00000000..70011ffc --- /dev/null +++ b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java @@ -0,0 +1,65 @@ +/** + * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ +package com.linkedin.kmf.common; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.Writer; +import java.lang.reflect.Field; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; + + +/** + * Generates the table of configuration parameters, their documentation strings and default values. + */ +public class ConfigDocumentationGenerator { + + private static void printHelp() { + System.out.println("ConfigDocumentationGenerator outputDirectory configClassNames..."); + } + + private static void printHtmlHeader(Writer out, String docClass) throws IOException { + out.write("Kafka Monitoring Automatically Generated Documentation. \n"); + out.write("

"); + out.write(docClass); + out.write("

\n"); + } + private static void printHtmlFooter(Writer out) throws IOException { + out.write("\n\n"); + } + + public static void main(String[] argv) throws Exception { + if (argv.length < 2) { + printHelp(); + System.exit(1); + } + + File outputDir = new File(argv[0]); + if (!outputDir.exists()) { + outputDir.mkdirs(); + } + + for (int i = 1; i < argv.length; i++) { + Class configClass = (Class) Class.forName(argv[i]); + Field configDefField = configClass.getDeclaredField("CONFIG"); + configDefField.setAccessible(true); + ConfigDef configDef = (ConfigDef) configDefField.get(null); + String docClass = configClass.getSimpleName(); + File outputFile = new File(outputDir, docClass + ".html"); + try (FileWriter fout = new FileWriter(outputFile)) { + printHtmlHeader(fout, docClass); + fout.write(configDef.toHtmlTable()); + printHtmlFooter(fout); + } + } + } +} diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index dc8676d0..75977d8c 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -231,11 +231,13 @@ static class TopicManagementHelper { private final int _minPartitionNum; private final TopicFactory _topicFactory; private final Properties _topicProperties; + private boolean _preferredLeaderElectionRequested; private int _requestTimeoutMs; private List _bootstrapServers; private final AdminClient _adminClient; + @SuppressWarnings("unchecked") TopicManagementHelper(Map props) throws Exception { TopicManagementServiceConfig config = new TopicManagementServiceConfig(props); @@ -247,6 +249,7 @@ static class TopicManagementHelper { _replicationFactor = config.getInt(TopicManagementServiceConfig.TOPIC_REPLICATION_FACTOR_CONFIG); _minPartitionsToBrokersRatio = config.getDouble(TopicManagementServiceConfig.PARTITIONS_TO_BROKERS_RATIO_CONFIG); _minPartitionNum = config.getInt(TopicManagementServiceConfig.MIN_PARTITION_NUM_CONFIG); + _preferredLeaderElectionRequested = false; _requestTimeoutMs = adminClientConfig.getInt(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG); _bootstrapServers = adminClientConfig.getList(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG); @@ -255,6 +258,7 @@ static class TopicManagementHelper { for (Map.Entry entry: ((Map) props.get(TopicManagementServiceConfig.TOPIC_PROPS_CONFIG)).entrySet()) _topicProperties.put(entry.getKey(), entry.getValue().toString()); } + Map topicFactoryConfig = props.containsKey(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) ? (Map) props.get(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) : new HashMap(); _topicFactory = (TopicFactory) Class.forName(topicFactoryClassName).getConstructor(Map.class).newInstance(topicFactoryConfig); From ce201ae3cc081e840c9449fc2e03e51d90f7d030 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 10 Dec 2019 18:43:53 -0800 Subject: [PATCH 058/192] Remove Static Import & Allow conciseness on Measurables using lamda expression (#179) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Remove Static wildcard Imports & Allow conciseness on Measurables using lamda expression 🎨 Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/KafkaMonitor.java | 9 +-- .../kmf/partitioner/NewKMPartitioner.java | 4 +- .../linkedin/kmf/services/ProduceService.java | 65 +++++++++---------- 3 files changed, 32 insertions(+), 46 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index 3c692d32..a158407c 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -24,7 +24,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; @@ -81,13 +80,7 @@ public KafkaMonitor(Map testProps) throws Exception { reporters.add(new JmxReporter(JMX_PREFIX)); Metrics metrics = new Metrics(new MetricConfig(), reporters, new SystemTime()); metrics.addMetric(metrics.metricName("offline-runnable-count", METRIC_GROUP_NAME, "The number of Service/App that are not fully running"), - new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - return _offlineRunnables.size(); - } - } - ); + (config, now) -> _offlineRunnables.size()); } public synchronized void start() { diff --git a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java index 08e0dd0d..348eb96c 100644 --- a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java +++ b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java @@ -9,13 +9,11 @@ */ package com.linkedin.kmf.partitioner; -import static org.apache.kafka.common.utils.Utils.murmur2; - public class NewKMPartitioner implements KMPartitioner { public int partition(String key, int partitionNum) { byte[] keyBytes = key.getBytes(); - return toPositive(murmur2(keyBytes)) % partitionNum; + return toPositive(org.apache.kafka.common.utils.Utils.murmur2(keyBytes)) % partitionNum; } private static int toPositive(int number) { diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 056de4c9..fcc131c8 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -40,7 +40,6 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; @@ -267,43 +266,39 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { new Percentile(new MetricName("produce-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile delay in ms for produce request", tags), 99.99))); metrics.addMetric(new MetricName("produce-availability-avg", METRIC_GROUP_NAME, "The average produce availability", tags), - new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - double availabilitySum = 0.0; - int partitionNum = _partitionNum.get(); - for (int partition = 0; partition < partitionNum; partition++) { - double recordsProduced = metrics.metrics().get(metrics.metricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); - double produceError = metrics.metrics().get(metrics.metricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); - // If there is no error, error rate sensor may expire and the value may be NaN. Treat NaN as 0 for error rate. - if (Double.isNaN(produceError) || Double.isInfinite(produceError)) { - produceError = 0; - } - // If there is either succeeded or failed produce to a partition, consider its availability as 0. - if (recordsProduced + produceError > 0) { - availabilitySum += recordsProduced / (recordsProduced + produceError); - } else if (!_treatZeroThroughputAsUnavailable) { - // If user configures treatZeroThroughputAsUnavailable to be false, a partition's availability - // is 1.0 as long as there is no exception thrown from producer. - // This allows kafka admin to exactly monitor the availability experienced by Kafka users which - // will block and retry for a certain amount of time based on its configuration (e.g. retries, retry.backoff.ms). - // Note that if it takes a long time for messages to be retries and sent, the latency in the ConsumeService - // will increase and it will reduce ConsumeAvailability if the latency exceeds consume.latency.sla.ms - // If timeout is set to more than 60 seconds (the current samples window duration), - // the error sample might be expired before the next error can be produced. - // In order to detect offline partition with high producer timeout config, the error status during last - // send is also checked before declaring 1.0 availability for the partition. - Boolean lastSendError = _produceErrorInLastSendPerPartition.get(partition); - if (lastSendError == null || !lastSendError) { - availabilitySum += 1.0; - } + (config, now) -> { + double availabilitySum = 0.0; + int partitionNum = _partitionNum.get(); + for (int partition = 0; partition < partitionNum; partition++) { + double recordsProduced = metrics.metrics().get(metrics.metricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); + double produceError = metrics.metrics().get(metrics.metricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); + // If there is no error, error rate sensor may expire and the value may be NaN. Treat NaN as 0 for error rate. + if (Double.isNaN(produceError) || Double.isInfinite(produceError)) { + produceError = 0; + } + // If there is either succeeded or failed produce to a partition, consider its availability as 0. + if (recordsProduced + produceError > 0) { + availabilitySum += recordsProduced / (recordsProduced + produceError); + } else if (!_treatZeroThroughputAsUnavailable) { + // If user configures treatZeroThroughputAsUnavailable to be false, a partition's availability + // is 1.0 as long as there is no exception thrown from producer. + // This allows kafka admin to exactly monitor the availability experienced by Kafka users which + // will block and retry for a certain amount of time based on its configuration (e.g. retries, retry.backoff.ms). + // Note that if it takes a long time for messages to be retries and sent, the latency in the ConsumeService + // will increase and it will reduce ConsumeAvailability if the latency exceeds consume.latency.sla.ms + // If timeout is set to more than 60 seconds (the current samples window duration), + // the error sample might be expired before the next error can be produced. + // In order to detect offline partition with high producer timeout config, the error status during last + // send is also checked before declaring 1.0 availability for the partition. + Boolean lastSendError = _produceErrorInLastSendPerPartition.get(partition); + if (lastSendError == null || !lastSendError) { + availabilitySum += 1.0; } } - // Assign equal weight to per-partition availability when calculating overall availability - return availabilitySum / partitionNum; } - } - ); + // Assign equal weight to per-partition availability when calculating overall availability + return availabilitySum / partitionNum; + }); } void addPartitionSensors(int partition) { From c50368a3f27d07e6879934d967baaf797c0c5772 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 11 Dec 2019 21:37:02 -0800 Subject: [PATCH 059/192] genericRecord in Utils class enable public access (#183) Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/common/Utils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index f8815d2b..4909d2d0 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -150,7 +150,7 @@ public static GenericRecord genericRecordFromJson(String message) { return record; } - private static String jsonFromGenericRecord(GenericRecord record) { + public static String jsonFromGenericRecord(GenericRecord record) { ByteArrayOutputStream out = new ByteArrayOutputStream(); GenericDatumWriter writer = new GenericDatumWriter<>(DefaultTopicSchema.MESSAGE_V0); From 5edfc4f988c6002e6c5a268fa9d8661b2409e2fb Mon Sep 17 00:00:00 2001 From: Alexandre Garnier Date: Thu, 12 Dec 2019 19:09:18 +0100 Subject: [PATCH 060/192] Create a tarball/zip of all necessary files (#159) Create a tarball/zip of all necessary files (#159) Signed-off-by: Andrew Choi --- build.gradle | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/build.gradle b/build.gradle index 423deac8..41467516 100644 --- a/build.gradle +++ b/build.gradle @@ -11,6 +11,7 @@ buildscript { } apply plugin: 'maven-publish' apply plugin: 'com.jfrog.bintray' +apply plugin: 'distribution' allprojects { @@ -97,6 +98,8 @@ allprojects { artifact sourceJar artifact javadocJar artifact testJar + artifact distZip + artifact distTar pom { name = 'kafka-monitor' @@ -129,6 +132,7 @@ allprojects { } task createConfigDocs( dependsOn : compileJava, type : JavaExec) { + outputs.dir configDocDir classpath sourceSets.main.runtimeClasspath main = 'com.linkedin.kmf.common.ConfigDocumentationGenerator' args = [configDocDir, @@ -152,6 +156,39 @@ allprojects { exceptionFormat = 'full' } } + + distributions { + main { + contents { + into('bin') { + from 'bin' + } + into('build/libs') { + from jar + } + into('build/dependant-libs') { + from copyDependantLibs + } + into('config') { + from 'config' + } + into('build/configDocs') { + from createConfigDocs + } + into('webapp') { + from 'webapp' + } + from('.') { + include 'README.md' + } + } + } + } + tasks.withType(Tar){ + compression = Compression.GZIP + extension = 'tar.gz' + } + } bintray { From daa6395deb9142b0ba92bd650e3bc72a9045cfd5 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 16 Dec 2019 10:29:23 -0800 Subject: [PATCH 061/192] Kafka Monitor Logo Design (#173) * kmf logo + kafka monitor logo red cross sign Signed-off-by: Andrew Choi --- docs/images/xinfra_monitor.png | Bin 0 -> 7938 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/images/xinfra_monitor.png diff --git a/docs/images/xinfra_monitor.png b/docs/images/xinfra_monitor.png new file mode 100644 index 0000000000000000000000000000000000000000..d0dfce610bfcf68739f14825d60c73549f8162e4 GIT binary patch literal 7938 zcmeHMXH=6}w>}|gL^KePYLKQPf)d1`2t$!3h%({`(g{@!ihxoB1XKtbF^*w?5n)C| z9i>Ak0x{BzR1xXY1E>%PAP@$GdlaGXGhrL*R6;^Fn5p0#qVaW; zFQ}~%w?9rq{_w=`NKXBol*MI*P3GHnJig$4!XW96eP)L=0EP)`1ycY}&8bLWs9=N! zAf8_bfSn>xHtLbT8vjzr|B*AK*%Kc=p#=zIT4dya*(FCpWzitz*S8M)w9i^9FJt_< zt)0SyxmXO?J=iDw9^=3C(k^?xJ{sS~uaj{O5xET$$02lLfScKr!tniwdxDoVji`l} z&wpZpqVjFLiimRA2Vm3ZA@x5?~Kxl$yMa*c=={)PU;+;UuXL%OC8_xp#evG zaA+a0_nYSzMYy|SQg5D-U#;7a0=V%*Rj;+GQeSQgSluR%2G$CVp;Wo?yG2qn_~TR4 zw!$E7GHK|yy&7Z0My1QM?Uu^b3A|}{EMW9-`=d^<4tqeon-d7@2PiICpK*4@CjuSW7~1%$?bYa6j)MejQn|KSR?6yE_9$|(N~;hdQXN>awJs8UZl-DRGE26?z#N={?? z?L>4@`4gvZ85GG>+RSdD@3I$TlDar_6G~s(bAEchktQ>HjUW-Q+P{1V7@}4`^0kB( z7&C~G9veyeAuZ|3x@T7tjl2n${9#rp3UOhqh zR%8Txaw88qOyn7V?j(n#^JslZMDIze+V{$%nlrX>#uV8OhntVU>&yiyISC67)nIzZ zxXP^q_CYL($FzV{;=qieI@++MC+-urnDDr+xYvP{BFt{MMB`;eYToAqYw@#LGq)SX z2UVo(?UNUj3f7JhZTIg)kZ#dzVi#*A1PHSdvX2aef>=pPb6UfT&;+)r*?I@iu)MK88&n!UCsbMme%3g{R1N=FO8F+;T*K^D~T zF#l3OG6o4cFl4MZ2)@H+kVvSV;L0z=LWM5}n zqWLc;&1^#`a;PJ3_xD#gF`IdZqI%17lL$NFrZxra@bKIq*Ln9^T}aW5lL#K4E*n_% zO!1}#UDu`;rY36ZGhR#Svi0iFI)-T(=#GA8JHRlcBH170S>YiGAnN*UG15*g-8($P zT=f;NX-5D4wX;>#Fu+K{nS6Vr#Wsc{%b@_tBnAtBh(G`UM|QFhK<@UxY5ca3r(?OU z7;S|Fh2c}T2ziFn3d4E|&F8X9D%Cau?!h3ID9xtdq~?^#%1Me^{nH?;+#dCffTv)j ze;;pGK?{(=7?{;u(6X*DsLucFuie2>lLtUZgspfMAB4fEm4{TB=ypil#{#Gv*;|s+ zZ4E(_2+#pLoSCa}uW(gsq9K70hV9rIlT~xw8e!$f+ zVnmo9jf&Ar2}FR?@~o1Uh`OaKU*3P;XB?(1PZaP{<>D~y4u08MfUEu|BCFoT3m>eh zN`d2->*=Ub$|DGGvG=>^~yJ%%SK^l zI9`Js zTphP2BtBRwGoJR&tfsa_|H(XK*qCfWn=PMzrry};UzVN;YWAx1+)*j3XqgU*vvf_g6ijB0|sN@;D6}g z)umi`TDxAL^p#lA`CHUi>(8~7rJLwLpF#WfL0N8+lWe8+CGW2=Z9tSTn`uig>s~9a z=L$%K@~z3UqR$7aSe>4K^L{fI1xFk!ul%LV|Fks~DZ44a?QTNWe!chY7Z-ntV+edQ zrXqXUj0Uc@L@-|rwrg*B6j%Fs5qt&Uz&|ze?+)D`>e-^I512vi4mWm<$^ZGbV+DF~ z5OA}1x=1fWZ-SXd5#V?UgBfsiJMFjbJD<(WzX}V}6Zyhlg@wA4UGk{n5AQx!U5Si* zzrAu#X2sd4UA#wixM%-u0bu-qub#q6`3Ku-xJQi zjF|~5&1?Et=+=4u`-$A}#re`s^~77)Y`w#|see58LV#7ho-uYq;V*CCx4^vpn+eA+g_9fJ=%~{+teCD!7i7ISsT5ip>e!w4+d|M0v zv(c(%>vaz?J4IOR{GNFD&*X!+*kId~x-za542_hbcYE8kCG-_p2$FiLY0bUFseWf& z2_6}yjN+V*uJHjeRhAP-^;XbAbZ9A zq?sdEzdS2nck_v!6_hzcx+<$9Oc>+m{J!i=b$)~{mY}|>4cC1D!#>pLmGF~l_2ofU zi%k@Rrb3++MFh#~xA#*jQ_RC9lYi!(zvk8VeZ*yfTe+q3H-YhlY_CzH-RqF(>mL78 zI_}$>_$3Fk<`{8w7|CL=gP0hsT#G4+u+~to2-XeB)|qhQwAk%H42r_PUcs11AqNY( ztW8ud_f)INz%(z1V$FNMn0U?}-TC1u-kn=!o?tAw3vslmQ_5$+*SZ{_>G^ zdN8gxBPElg#|N;xvJJgOxneM@hwZac_A0Gxz3`^+x15fTS^K3!fe4) zmQpTP)kM?#CLAWZ>odUXXz>vVi}I0aLN;oh5SiKjIB{?(<@5KJI(~(L;Pp+LdW&AE zmS<@yU;V3Ox)NlG zK1q-zx&;&KAbWEP$EST-XQ3CB+=-BI2D|eu?t)>eI^v~IGdk2DA>0*!(|e|8G$4I$ zokFbU_IDzH_?WByxW_6b>(T?~!BCz2Fm^}*-I&mlE;E}@Br$nndIA7J|DAP7 zM%3VkpY12&)(3h|p)&S1;5{=>X42-!yzuo-8lR44x4TwIm{5bCG~ID66;l9>^~+@K zgHH-nV=6o{X`h2`KeC?TCw(X2_6`-1rrTiIhDk(yP$E{tSDZ#N}C7MZlem zoH}d`j$H24=Z#n4PwN{sU$?0@D&KV<_MG|bk9d?ab6nCMZ@P@Q_QUickyO6#LQB+3r|fHj!TJsJ~**No+@wau@b19KWg^m1o>+ zM<^eS<6OC#lPw4IUR_@PS+?XLTND;N+ANj*Vh29;m0&@K^pr}Z10kYt>aHBCn`~1* zH*}vSKBktn7ZAHJvSxZMkazreC2muIW)3aj-S9h#4)`{#Q(>jCbzGTW$MK|Px1a8q zc~BGLbSYkIrQ+s8o|`P=x}488PP6JuPMbkr1D3Q)a!Nk3HK^M8!7D9MHYKO&u558G zGGiZTtifFm=vG*%a?-x5z)HDVe1$Q<_c?UguW0RAxFGN-CA^T~JTKuqw+mu+@BW=V z5OKo)QF;jTo{26~$;6u1K(IgLHY=7BH}|k3%)6CjrGH2^IUAbHB-N}&#r(Fxkh4kZ zTL-m$3e*>@%{bjm+IS{Sz9vd^#Fu#y{aG{hWObRBQFMN{Jjy$h79ff7fw1cbJY}@B zBMcuMoev?H_fUrOjBBm7?4oYx@?oncTvfnnLgi&o z(y;>td{&lsB_u|yT4fbgQ3>cgiy&6?E8d>EU>UL9lQ|q(d6#I{Cb-5079HFZn|i#- z%DHy?yyxV|KO@`rN-Q z_$==kH@cILEBD_8$^Mm&j(Qm7ckRQw-Co<+E`9NK|NG$ZfL zrT5P|o=r26xAKkAl%boa9-eHD7`y?oTE?DCC$pNPPmh_$cF!PDSe^*n`e|5CXg=2# zUmnzqBtk|6b$-C#m;Rw9U-v1nUI*WA9bnyoF6J}jQW7XLXu%IH>M&t%vQo|4EO;=r zZSBmsHuFx|W6>igETt}N2!`QL+?@xq6g~DbOY)4-4GeF5H+IX53;$^L_m z&=9+Sp&fAIhPuB@O_2<$v4*ui7;2&|&St1qbB581?yf&;V2nvZx`Ptcj}!YYy50Hx z>g9m(|Mb!UR-@w1DzBe-{Bqb37bun`kQ>O_vJ4o>#CO!4IG6pS^bfGIY+hrr+;drv zh zdAnk;KNM+|re_)8kOuJqT{{$2s6XHQb;X+#ow9UyMyyLumyIh3LX4MPMtQ%l_`C(F zd4yWcJKjh^YRunX|75W?Qyz0voy+L*IFDdhMrP+31+nl>=5U!ZXNiz1Z-D^G&fD`F z#Rp#-U2Vd!aXH9|t*RusxUEn(ihDCdULbCK2bs%G-&mOdF!(glW+7YJHx`lwMs4<~ zD^0M5rU+w&>b2WqT`)=fyV9ZJjJz}~@=U?G17ri-l;6c+HD982C=)$lNWnoLDPO8w zoWHY!pBrLD^~zQeC{EzIIO$$_M&2x#nQ6V`F8UPHs?W4064vPvSDTRHH#3)# zbZ2HPY_cJID>}-ws<>cKNZN-p{@qL*ad+?f=wHh&ck(p4w< z26KGgL`%kT3XRdCZ>E?5ZyD&xpOA{uLDg!pY9q?H{l6y@Sl9ULzVENQ&i{>jUe9>) a8uEpi`;xNPOUQ2kurNJtl5^NC=Dz@Q*-3E# literal 0 HcmV?d00001 From 6aa913941c22ac3e82f6a463d626ec8a9afc6246 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 16 Dec 2019 15:34:44 -0800 Subject: [PATCH 062/192] Checkstyle Revisions (#182) checkstyle revisions for nolinewrapp Signed-off-by: Andrew Choi --- checkstyle/checkstyle.xml | 4 +++- config/kafka-monitor.properties | 2 -- src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java | 2 +- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 2 +- .../linkedin/kmf/services/configs/CommonServiceConfig.java | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index e2be1a6a..63736f49 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -26,7 +26,6 @@ - @@ -36,6 +35,7 @@ + @@ -81,4 +81,6 @@ + + diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 540f24a3..7803d0f3 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -145,5 +145,3 @@ # } } - - diff --git a/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java b/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java index e2ecade6..2ebc6ca0 100644 --- a/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java +++ b/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java @@ -23,4 +23,4 @@ public interface KMBaseProducer { void close(); -} \ No newline at end of file +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 0475519b..afd87e18 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -280,4 +280,4 @@ private class ConsumeMetrics { }); } } -} \ No newline at end of file +} diff --git a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java index 654685a6..46316c01 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java @@ -31,4 +31,4 @@ public class CommonServiceConfig { public static final String REPORT_INTERVAL_SEC_CONFIG = "report.interval.sec"; public static final String REPORT_INTERVAL_SEC_DOC = "The interval in second by which metrics reporter service will report the metrics values."; -} \ No newline at end of file +} From 1e69b9e3939406e6ff633b4637851441a1be12c5 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 17 Dec 2019 13:31:35 -0800 Subject: [PATCH 063/192] Update README Markdown with xinfra health monitor logo (#185) Update README Markdown File with Xinfra health monitor logo Signed-off-by: Andrew Choi --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index cef3c128..f4e56110 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,5 @@ # Kafka Monitor +drawing [![Build Status](https://travis-ci.org/linkedin/kafka-monitor.svg?branch=master)](https://travis-ci.org/linkedin/kafka-monitor) From f1a21c9a16df1d274f424111ae0fab69e6f85d0b Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 18 Dec 2019 11:09:31 -0800 Subject: [PATCH 064/192] Correct the ordering of metricConfig configuration in Consume Service and make MetricConfig execute after topic is ready (#181) Correct the ordering of metricConfig configuration in Consume Service and make MetricConfig execute after topic is ready: The original source code has the consume() thread starting running before the MetricConfig is configured. Thus, this PR corrects the ordering of metricConfig configuration within the Consume Service. Additionally, this PR enables MetricConfig to execute after topic and its respective topic partitions are ready. Tested locally. Signed-off-by: Andrew Choi --- config/kafka-monitor.properties | 7 +- config/multi-cluster-monitor.properties | 14 ++-- .../kmf/apps/MultiClusterMonitor.java | 25 ++++--- .../kmf/apps/SingleClusterMonitor.java | 12 +++- .../linkedin/kmf/services/ConsumeService.java | 72 +++++++++++-------- .../MultiClusterTopicManagementService.java | 25 ++++--- .../kmf/services/TopicManagementService.java | 6 ++ .../linkedin/kmf/tests/BasicEndToEndTest.java | 3 +- 8 files changed, 94 insertions(+), 70 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 7803d0f3..e0b4efe3 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -25,7 +25,7 @@ # }, # ... # } -# + # TestClassName can be canonical name or simple name of any class that implements # interface com.linkedin.kmf.services.Test. These classes should be under # package com.linkedin.kmf.tests. @@ -38,7 +38,7 @@ # or ServiceClassName. The key for the test/service in the json map is used as name to # identify the test/service in the log or JMX metrics, which is useful if multiple # test/service with the same class.name are run in the same Kafka Monitor process. -# + { "single-cluster-monitor": { @@ -76,7 +76,6 @@ "class.name": "com.linkedin.kmf.services.JolokiaService" }, - "reporter-service": { "class.name": "com.linkedin.kmf.services.DefaultMetricsReporterService", "report.interval.sec": 1, @@ -96,7 +95,6 @@ ] } - # Example statsd-service to report metrics # "statsd-service": { # "class.name": "com.linkedin.kmf.services.StatsdMetricsReporterService", @@ -111,7 +109,6 @@ # ] # } - # Example kafka-service to report metrics # "reporter-kafka-service": { # "class.name": "com.linkedin.kmf.services.KafkaMetricsReporterService", diff --git a/config/multi-cluster-monitor.properties b/config/multi-cluster-monitor.properties index cc1ce37a..4c8ef947 100644 --- a/config/multi-cluster-monitor.properties +++ b/config/multi-cluster-monitor.properties @@ -12,12 +12,13 @@ # each cluster in the pipeline. The "produce.service.props" should use the first cluster and # the "consume.service.props" should use the last cluster in the pipeline. +# Produce service: Configure Produce Service to produce to the first cluster of the pipeline +# Consume service: Configure Consume Service to consume from the last cluster of the pipeline +# Last cluster: If there are more than two clusters in the pipeline, add one property map for each one of them. { "multi-cluster-monitor": { "class.name": "com.linkedin.kmf.apps.MultiClusterMonitor", "topic": "kafka-monitor-topic", - - # Configure Produce Service to produce to the first cluster of the pipeline "produce.service.props": { "zookeeper.connect": "localhost:2181/first_cluster", "bootstrap.servers": "localhost:9092", @@ -26,8 +27,6 @@ "client.id": "kafka-monitor-client-id" } }, - - # Configure Consume Service to consume from the last cluster of the pipeline "consume.service.props": { "zookeeper.connect": "localhost:2181/last_cluster", "bootstrap.servers": "localhost:9095", @@ -39,6 +38,7 @@ "topic.management.props.per.cluster" : { "first-cluster" : { + "bootstrap.servers": "localhost:9092", "zookeeper.connect": "localhost:2181/first_cluster", "topic-management.topicCreationEnabled": true, "topic-management.replicationFactor" : 1, @@ -48,10 +48,8 @@ } }, - # If there are more than two clusters in the pipeline, - # add one property map for each one of them. - "last-cluster" : { + "bootstrap.servers": "localhost:9095", "zookeeper.connect": "localhost:2181/last_cluster", "topic-management.topicCreationEnabled": true, "topic-management.replicationFactor" : 1, @@ -91,5 +89,3 @@ "class.name": "com.linkedin.kmf.services.JolokiaService" } } - - diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index 0ae0332c..25e7b9e7 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -15,6 +15,7 @@ import com.linkedin.kmf.services.ProduceService; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,7 +30,7 @@ public class MultiClusterMonitor implements App { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterMonitor.class); - private final MultiClusterTopicManagementService _topicManagementService; + private final MultiClusterTopicManagementService _multiClusterTopicManagementService; private final ProduceService _produceService; private final ConsumeService _consumeService; private final String _name; @@ -37,9 +38,10 @@ public class MultiClusterMonitor implements App { public MultiClusterMonitor(Map props, String name) throws Exception { _name = name; MultiClusterMonitorConfig config = new MultiClusterMonitorConfig(props); - _topicManagementService = new MultiClusterTopicManagementService(createMultiClusterTopicManagementServiceProps(props, config), name); + _multiClusterTopicManagementService = new MultiClusterTopicManagementService(createMultiClusterTopicManagementServiceProps(props, config), name); + CompletableFuture topicPartitionReady = _multiClusterTopicManagementService.topicPartitionReady(); _produceService = new ProduceService(createProduceServiceProps(props, config), name); - _consumeService = new ConsumeService(createConsumeServiceProps(props, config), name); + _consumeService = new ConsumeService(createConsumeServiceProps(props, config), name, topicPartitionReady); } @SuppressWarnings("unchecked") @@ -70,15 +72,18 @@ private Map createMultiClusterTopicManagementServiceProps(Map completableFuture = _multiClusterTopicManagementService.topicManagementReady(); + completableFuture.thenRun(() -> { + _produceService.start(); + _consumeService.start(); + }); + LOG.info(_name + "/MultiClusterMonitor started."); } @Override public void stop() { - _topicManagementService.stop(); + _multiClusterTopicManagementService.stop(); _produceService.stop(); _consumeService.stop(); LOG.info(_name + "/MultiClusterMonitor stopped"); @@ -86,12 +91,12 @@ public void stop() { @Override public boolean isRunning() { - return _topicManagementService.isRunning() && _produceService.isRunning() && _consumeService.isRunning(); + return _multiClusterTopicManagementService.isRunning() && _produceService.isRunning() && _consumeService.isRunning(); } @Override public void awaitShutdown() { - _topicManagementService.awaitShutdown(); + _multiClusterTopicManagementService.awaitShutdown(); _produceService.awaitShutdown(); _consumeService.awaitShutdown(); } diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 46a099fa..a38ee22a 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -52,8 +52,9 @@ public class SingleClusterMonitor implements App { public SingleClusterMonitor(Map props, String name) throws Exception { _name = name; _topicManagementService = new TopicManagementService(props, name); + CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionReady(); _produceService = new ProduceService(props, name); - _consumeService = new ConsumeService(props, name); + _consumeService = new ConsumeService(props, name, topicPartitionReady); } @Override @@ -77,6 +78,15 @@ public void stop() { @Override public boolean isRunning() { + if (!_topicManagementService.isRunning()) { + LOG.info("_topicManagementService not running."); + } + if (!_produceService.isRunning()) { + LOG.info("_produceService not running."); + } + if (!_consumeService.isRunning()) { + LOG.info("_consumeService not Running."); + } return _topicManagementService.isRunning() && _produceService.isRunning() && _consumeService.isRunning(); } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index afd87e18..bb5e665b 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.Properties; import java.util.Random; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -57,16 +58,16 @@ public class ConsumeService implements Service { ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; private final String _name; - private final ConsumeMetrics _sensors; + private ConsumeMetrics _sensors; private final KMBaseConsumer _consumer; - private final Thread _thread; + private Thread _thread; private final int _latencyPercentileMaxMs; private final int _latencyPercentileGranularityMs; private final AtomicBoolean _running; private final int _latencySlaMs; - private final AdminClient _adminClient; + private AdminClient _adminClient; - public ConsumeService(Map props, String name) throws Exception { + public ConsumeService(Map props, String name, CompletableFuture topicPartitionReady) throws Exception { _name = name; Map consumerPropsOverride = props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) ? (Map) props.get(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) : new HashMap<>(); @@ -84,7 +85,6 @@ public ConsumeService(Map props, String name) throws Exception { throw new ConfigException("Override must not contain " + property + " config."); } } - Properties consumerProps = new Properties(); // Assign default config. This has the lowest priority. @@ -94,7 +94,6 @@ public ConsumeService(Map props, String name) throws Exception { consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "kmf-consumer-group-" + new Random().nextInt()); consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - if (consumerClassName.equals(NewConsumer.class.getCanonicalName()) || consumerClassName.equals(NewConsumer.class.getSimpleName())) { consumerClassName = NewConsumer.class.getCanonicalName(); } @@ -105,25 +104,26 @@ public ConsumeService(Map props, String name) throws Exception { // Assign config specified for consumer. This has the highest priority. consumerProps.putAll(consumerPropsOverride); - _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); - _thread = new Thread(() -> { - try { - consume(); - } catch (Exception e) { - LOG.error(_name + "/ConsumeService failed", e); - } - }, _name + " consume-service"); - _thread.setDaemon(true); - - MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); - List reporters = new ArrayList<>(); - reporters.add(new JmxReporter(JMX_PREFIX)); - Metrics metrics = new Metrics(metricConfig, reporters, new SystemTime()); - Map tags = new HashMap<>(); - tags.put("name", _name); - _adminClient = AdminClient.create(props); - _sensors = new ConsumeMetrics(metrics, tags, topic); + topicPartitionReady.thenRun(() -> { + MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); + List reporters = new ArrayList<>(); + reporters.add(new JmxReporter(JMX_PREFIX)); + Metrics metrics = new Metrics(metricConfig, reporters, new SystemTime()); + Map tags = new HashMap<>(); + tags.put("name", _name); + _adminClient = AdminClient.create(props); + _sensors = new ConsumeMetrics(metrics, tags, topic, topicPartitionReady); + _thread = new Thread(() -> { + try { + consume(); + } catch (Exception e) { + LOG.error(_name + "/ConsumeService failed", e); + } + }, _name + " consume-service"); + _thread.setDaemon(true); + }); + } private void consume() throws Exception { @@ -221,13 +221,23 @@ private class ConsumeMetrics { private final Sensor _recordsDelay; private final Sensor _recordsDelayed; - ConsumeMetrics(final Metrics metrics, final Map tags, String topicName) throws ExecutionException, InterruptedException { - int partitionCount = 0; - DescribeTopicsResult describeTopicsResult = _adminClient.describeTopics(Collections.singleton(topicName)); - Map> values = describeTopicsResult.values(); - partitionCount = values.get(topicName).get().partitions().size(); - Sensor topicPartitionCount = metrics.sensor("topic-partitions"); - topicPartitionCount.add(new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), new Total(partitionCount)); + ConsumeMetrics(final Metrics metrics, final Map tags, String topicName, CompletableFuture topicPartitionReady) { + topicPartitionReady.thenRun(() -> { + DescribeTopicsResult describeTopicsResult = _adminClient.describeTopics(Collections.singleton(topicName)); + Map> topicResultValues = describeTopicsResult.values(); + KafkaFuture topicDescriptionKafkaFuture = topicResultValues.get(topicName); + TopicDescription topicDescription = null; + try { + topicDescription = topicDescriptionKafkaFuture.get(); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Exception occurred while retrieving the topic description.", e); + } + int partitionCount = topicDescription.partitions().size(); + Sensor topicPartitionCount = metrics.sensor("topic-partitions"); + topicPartitionCount.add( + new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), + new Total(partitionCount)); + }); _bytesConsumed = metrics.sensor("bytes-consumed"); _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 75977d8c..530b0b13 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -28,7 +28,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import kafka.admin.AdminUtils; @@ -57,7 +56,7 @@ /** - * This service periodically checks and rebalances the monitor topics across a pipeline of Kafka clusters so that + * This service periodically checks and re-balances the monitor topics across a pipeline of Kafka clusters so that * leadership of the partitions of the monitor topic in each cluster is distributed evenly across brokers in the cluster. * * More specifically, this service may do some or all of the following tasks depending on the config: @@ -66,21 +65,23 @@ * - Increase partition number of the monitor topic if either partitionsToBrokersRatio or minPartitionNum is not satisfied * - Increase replication factor of the monitor topic if the user-specified replicationFactor is not satisfied * - Reassign partition across brokers to make sure each broker acts as preferred leader of at least one partition of the monitor topic - * - Trigger preferred leader election to make sure each broker acts as leader of at least one partition of the monitor topic. + * - Trigger preferred leader election to make sure each broker acts as the leader of at least one partition of the monitor topic. * - Make sure the number of partitions of the monitor topic is same across all monitored clusters. * */ public class MultiClusterTopicManagementService implements Service { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); private static final String METRIC_GROUP_NAME = "topic-management-service"; + private final CompletableFuture _topicPartitionReady = new CompletableFuture<>(); private final AtomicBoolean _isRunning = new AtomicBoolean(false); private final String _serviceName; private final Map _topicManagementByCluster; private final int _scheduleIntervalMs; private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; - final private CompletableFuture _completableFuture; + private final CompletableFuture _completableFuture; + @SuppressWarnings("unchecked") public MultiClusterTopicManagementService(Map props, String serviceName) throws Exception { _serviceName = serviceName; MultiClusterTopicManagementServiceConfig config = new MultiClusterTopicManagementServiceConfig(props); @@ -91,18 +92,19 @@ public MultiClusterTopicManagementService(Map props, String serv _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); _completableFuture = new CompletableFuture<>(); - _executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - return new Thread(r, _serviceName + "-multi-cluster-topic-management-service"); - } - }); + _executor = Executors.newSingleThreadScheduledExecutor( + r -> new Thread(r, _serviceName + "-multi-cluster-topic-management-service")); + _topicPartitionReady.complete(null); } public CompletableFuture topicManagementReady() { return _completableFuture; } + public CompletableFuture topicPartitionReady() { + return _topicPartitionReady; + } + private Map initializeTopicManagementHelper(Map propsByCluster, String topic) throws Exception { Map topicManagementByCluster = new HashMap<>(); for (Map.Entry entry: propsByCluster.entrySet()) { @@ -177,7 +179,6 @@ public void run() { helper.maybeAddPartitions(minPartitionNum); } _completableFuture.complete(null); - for (Map.Entry entry : _topicManagementByCluster.entrySet()) { String clusterName = entry.getKey(); TopicManagementHelper helper = entry.getValue(); @@ -231,7 +232,6 @@ static class TopicManagementHelper { private final int _minPartitionNum; private final TopicFactory _topicFactory; private final Properties _topicProperties; - private boolean _preferredLeaderElectionRequested; private int _requestTimeoutMs; private List _bootstrapServers; @@ -249,7 +249,6 @@ static class TopicManagementHelper { _replicationFactor = config.getInt(TopicManagementServiceConfig.TOPIC_REPLICATION_FACTOR_CONFIG); _minPartitionsToBrokersRatio = config.getDouble(TopicManagementServiceConfig.PARTITIONS_TO_BROKERS_RATIO_CONFIG); _minPartitionNum = config.getInt(TopicManagementServiceConfig.MIN_PARTITION_NUM_CONFIG); - _preferredLeaderElectionRequested = false; _requestTimeoutMs = adminClientConfig.getInt(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG); _bootstrapServers = adminClientConfig.getList(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG); diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index cf8f1791..b41d3c24 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -23,10 +23,16 @@ */ public class TopicManagementService implements Service { private final MultiClusterTopicManagementService _multiClusterTopicManagementService; + CompletableFuture _topicPartitionReady = new CompletableFuture<>(); public TopicManagementService(Map props, String serviceName) throws Exception { Map serviceProps = createMultiClusterTopicManagementServiceProps(props, serviceName); _multiClusterTopicManagementService = new MultiClusterTopicManagementService(serviceProps, serviceName); + _topicPartitionReady.complete(null); + } + + public CompletableFuture topicPartitionReady() { + return _topicPartitionReady; } /** diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index f6f03c6b..af2cc1a0 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -44,8 +44,9 @@ public class BasicEndToEndTest implements Test { public BasicEndToEndTest(Map props, String name) throws Exception { _name = name; _topicManagementService = new TopicManagementService(props, name); + CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionReady(); _produceService = new ProduceService(props, name); - _consumeService = new ConsumeService(props, name); + _consumeService = new ConsumeService(props, name, topicPartitionReady); } @Override From c1b34dc554de155859bb08b48cc498ed16f02553 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 18 Dec 2019 16:32:24 -0800 Subject: [PATCH 065/192] [WIP] Enable SSL Configurations for ISSUE #178 (#187) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This will be merged after the PR (https://github.com/linkedin/kafka-monitor/pull/181) is merged. and then this PR rebased. Addresses the issue: ISSUE #178 🖌 Signed-off-by: Andrew Choi --- .../linkedin/kmf/apps/MultiClusterMonitor.java | 4 ++-- .../linkedin/kmf/apps/SingleClusterMonitor.java | 10 ++++++++-- .../com/linkedin/kmf/services/ConsumeService.java | 15 +++++++++------ .../MultiClusterTopicManagementService.java | 14 ++++++++------ .../com/linkedin/kmf/services/ProduceService.java | 11 ++++++++--- .../services/configs/ConsumeServiceConfig.java | 2 +- 6 files changed, 36 insertions(+), 20 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index 25e7b9e7..8e820374 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -73,8 +73,8 @@ private Map createMultiClusterTopicManagementServiceProps(Map completableFuture = _multiClusterTopicManagementService.topicManagementReady(); - completableFuture.thenRun(() -> { + CompletableFuture topicManagementReady = _multiClusterTopicManagementService.topicManagementReady(); + topicManagementReady.thenRun(() -> { _produceService.start(); _consumeService.start(); }); diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index a38ee22a..9edd5cb3 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -52,7 +52,9 @@ public class SingleClusterMonitor implements App { public SingleClusterMonitor(Map props, String name) throws Exception { _name = name; _topicManagementService = new TopicManagementService(props, name); + CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionReady(); + _produceService = new ProduceService(props, name); _consumeService = new ConsumeService(props, name, topicPartitionReady); } @@ -78,16 +80,21 @@ public void stop() { @Override public boolean isRunning() { + boolean isRunning = true; + if (!_topicManagementService.isRunning()) { + isRunning = false; LOG.info("_topicManagementService not running."); } if (!_produceService.isRunning()) { + isRunning = false; LOG.info("_produceService not running."); } if (!_consumeService.isRunning()) { + isRunning = false; LOG.info("_consumeService not Running."); } - return _topicManagementService.isRunning() && _produceService.isRunning() && _consumeService.isRunning(); + return isRunning; } @Override @@ -279,7 +286,6 @@ public static void main(String[] args) throws Exception { props.put(TopicManagementServiceConfig.TOPIC_REPLICATION_FACTOR_CONFIG, res.getInt("replicationFactor")); if (res.getInt("rebalanceMs") != null) props.put(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG, res.getInt("rebalanceMs")); - SingleClusterMonitor app = new SingleClusterMonitor(props, "single-cluster-monitor"); app.start(); diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index bb5e665b..6088088c 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -60,7 +60,7 @@ public class ConsumeService implements Service { private final String _name; private ConsumeMetrics _sensors; private final KMBaseConsumer _consumer; - private Thread _thread; + private Thread _consumeThread; private final int _latencyPercentileMaxMs; private final int _latencyPercentileGranularityMs; private final AtomicBoolean _running; @@ -104,6 +104,10 @@ public ConsumeService(Map props, String name, CompletableFuture< // Assign config specified for consumer. This has the highest priority. consumerProps.putAll(consumerPropsOverride); + + if (props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG)) { + props.forEach(consumerProps::putIfAbsent); + } _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); topicPartitionReady.thenRun(() -> { MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); @@ -114,16 +118,15 @@ public ConsumeService(Map props, String name, CompletableFuture< tags.put("name", _name); _adminClient = AdminClient.create(props); _sensors = new ConsumeMetrics(metrics, tags, topic, topicPartitionReady); - _thread = new Thread(() -> { + _consumeThread = new Thread(() -> { try { consume(); } catch (Exception e) { LOG.error(_name + "/ConsumeService failed", e); } }, _name + " consume-service"); - _thread.setDaemon(true); + _consumeThread.setDaemon(true); }); - } private void consume() throws Exception { @@ -185,7 +188,7 @@ record = _consumer.receive(); @Override public synchronized void start() { if (_running.compareAndSet(false, true)) { - _thread.start(); + _consumeThread.start(); LOG.info("{}/ConsumeService started.", _name); } } @@ -209,7 +212,7 @@ public void awaitShutdown() { @Override public boolean isRunning() { - return _running.get() && _thread.isAlive(); + return _running.get() && _consumeThread.isAlive(); } private class ConsumeMetrics { diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 530b0b13..95cef3cc 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -79,7 +79,9 @@ public class MultiClusterTopicManagementService implements Service { private final int _scheduleIntervalMs; private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; - private final CompletableFuture _completableFuture; + + private final CompletableFuture _topicManagementReady; + @SuppressWarnings("unchecked") public MultiClusterTopicManagementService(Map props, String serviceName) throws Exception { @@ -91,14 +93,14 @@ public MultiClusterTopicManagementService(Map props, String serv _topicManagementByCluster = initializeTopicManagementHelper(propsByCluster, topic); _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); - _completableFuture = new CompletableFuture<>(); + _topicManagementReady = new CompletableFuture<>(); _executor = Executors.newSingleThreadScheduledExecutor( r -> new Thread(r, _serviceName + "-multi-cluster-topic-management-service")); _topicPartitionReady.complete(null); } public CompletableFuture topicManagementReady() { - return _completableFuture; + return _topicManagementReady; } public CompletableFuture topicPartitionReady() { @@ -178,7 +180,7 @@ public void run() { for (TopicManagementHelper helper : _topicManagementByCluster.values()) { helper.maybeAddPartitions(minPartitionNum); } - _completableFuture.complete(null); + _topicManagementReady.complete(null); for (Map.Entry entry : _topicManagementByCluster.entrySet()) { String clusterName = entry.getKey(); TopicManagementHelper helper = entry.getValue(); @@ -261,7 +263,9 @@ static class TopicManagementHelper { Map topicFactoryConfig = props.containsKey(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) ? (Map) props.get(TopicManagementServiceConfig.TOPIC_FACTORY_PROPS_CONFIG) : new HashMap(); _topicFactory = (TopicFactory) Class.forName(topicFactoryClassName).getConstructor(Map.class).newInstance(topicFactoryConfig); + _adminClient = constructAdminClient(props); + LOG.info("{} configs: {}", _adminClient.getClass().getSimpleName(), props); } @SuppressWarnings("unchecked") @@ -277,8 +281,6 @@ void maybeCreateTopic() throws Exception { } AdminClient constructAdminClient(Map props) { - props.putIfAbsent(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, _bootstrapServers); - props.putIfAbsent(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, _requestTimeoutMs); return AdminClient.create(props); } diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index fcc131c8..3a3ec6dd 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -113,6 +113,7 @@ public ProduceService(Map props, String name) throws Exception { throw new ConfigException("Override must not contain " + property + " config."); } } + _adminClient = AdminClient.create(props); if (producerClass.equals(NewProducer.class.getCanonicalName()) || producerClass.equals(NewProducer.class.getSimpleName())) { @@ -121,7 +122,7 @@ public ProduceService(Map props, String name) throws Exception { _producerClassName = producerClass; } - initializeProducer(); + initializeProducer(props); _produceExecutor = Executors.newScheduledThreadPool(_threadsNum, new ProduceServiceThreadFactory()); _handleNewPartitionsExecutor = Executors.newSingleThreadScheduledExecutor(new HandleNewPartitionsThreadFactory()); @@ -135,7 +136,7 @@ public ProduceService(Map props, String name) throws Exception { _sensors = new ProduceMetrics(metrics, tags); } - private void initializeProducer() throws Exception { + private void initializeProducer(Map props) throws Exception { Properties producerProps = new Properties(); // Assign default config. This has the lowest priority. producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); @@ -151,6 +152,10 @@ private void initializeProducer() throws Exception { // Assign config specified for producer. This has the highest priority. producerProps.putAll(_producerPropsOverride); + if (props.containsKey(ProduceServiceConfig.PRODUCER_PROPS_CONFIG)) { + props.forEach(producerProps::putIfAbsent); + } + _producer = (KMBaseProducer) Class.forName(_producerClassName).getConstructor(Properties.class).newInstance(producerProps); LOG.info("{}/ProduceService is initialized.", _name); } @@ -380,7 +385,7 @@ public void run() { } _producer.close(); try { - initializeProducer(); + initializeProducer(new HashMap<>()); } catch (Exception e) { LOG.error("Failed to restart producer.", e); throw new IllegalStateException(e); diff --git a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java index 6a003c3c..a15ee61c 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java @@ -14,6 +14,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; + public class ConsumeServiceConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -79,7 +80,6 @@ public class ConsumeServiceConfig extends AbstractConfig { 20000, ConfigDef.Importance.MEDIUM, LATENCY_SLA_MS_DOC); - } public ConsumeServiceConfig(Map props) { From d01ca7f228676684d8509263a4f1a2dfb889e69b Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 19 Dec 2019 10:18:04 -0800 Subject: [PATCH 066/192] Appropriate variable naming conventions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🕍 Appropriate variable naming conventions such as CompletableFuture topicPartitionReady -> = _multiClusterTopicManagementService.topicPartitionResult 🕍 --- .../kmf/apps/MultiClusterMonitor.java | 4 ++-- .../kmf/apps/SingleClusterMonitor.java | 5 ++--- .../linkedin/kmf/services/ConsumeService.java | 19 ++++++++++--------- .../MultiClusterTopicManagementService.java | 18 +++++++++--------- .../kmf/services/TopicManagementService.java | 12 ++++++------ .../linkedin/kmf/tests/BasicEndToEndTest.java | 4 ++-- 6 files changed, 31 insertions(+), 31 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index 8e820374..04cbb650 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -39,7 +39,7 @@ public MultiClusterMonitor(Map props, String name) throws Except _name = name; MultiClusterMonitorConfig config = new MultiClusterMonitorConfig(props); _multiClusterTopicManagementService = new MultiClusterTopicManagementService(createMultiClusterTopicManagementServiceProps(props, config), name); - CompletableFuture topicPartitionReady = _multiClusterTopicManagementService.topicPartitionReady(); + CompletableFuture topicPartitionReady = _multiClusterTopicManagementService.topicPartitionResult(); _produceService = new ProduceService(createProduceServiceProps(props, config), name); _consumeService = new ConsumeService(createConsumeServiceProps(props, config), name, topicPartitionReady); } @@ -73,7 +73,7 @@ private Map createMultiClusterTopicManagementServiceProps(Map topicManagementReady = _multiClusterTopicManagementService.topicManagementReady(); + CompletableFuture topicManagementReady = _multiClusterTopicManagementService.topicManagementResult(); topicManagementReady.thenRun(() -> { _produceService.start(); _consumeService.start(); diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 9edd5cb3..3ce1dda2 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -53,8 +53,7 @@ public SingleClusterMonitor(Map props, String name) throws Excep _name = name; _topicManagementService = new TopicManagementService(props, name); - CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionReady(); - + CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionResult(); _produceService = new ProduceService(props, name); _consumeService = new ConsumeService(props, name, topicPartitionReady); } @@ -62,7 +61,7 @@ public SingleClusterMonitor(Map props, String name) throws Excep @Override public void start() { _topicManagementService.start(); - CompletableFuture completableFuture = _topicManagementService.topicManagementReady(); + CompletableFuture completableFuture = _topicManagementService.topicManagementResult(); completableFuture.thenRun(() -> { _produceService.start(); _consumeService.start(); diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 6088088c..e1898af6 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -14,6 +14,7 @@ import com.linkedin.kmf.consumer.BaseConsumerRecord; import com.linkedin.kmf.consumer.KMBaseConsumer; import com.linkedin.kmf.consumer.NewConsumer; +import com.linkedin.kmf.services.configs.CommonServiceConfig; import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import java.util.ArrayList; import java.util.Collections; @@ -53,10 +54,10 @@ public class ConsumeService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ConsumeService.class); private static final String METRIC_GROUP_NAME = "consume-service"; + private static final String TAGS_NAME = "name"; + private static final String FALSE = "false"; private static final String[] NON_OVERRIDABLE_PROPERTIES = - new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, - ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; - + new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; private final String _name; private ConsumeMetrics _sensors; private final KMBaseConsumer _consumer; @@ -67,7 +68,7 @@ public class ConsumeService implements Service { private final int _latencySlaMs; private AdminClient _adminClient; - public ConsumeService(Map props, String name, CompletableFuture topicPartitionReady) throws Exception { + public ConsumeService(Map props, String name, CompletableFuture topicPartitionResult) throws Exception { _name = name; Map consumerPropsOverride = props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) ? (Map) props.get(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) : new HashMap<>(); @@ -88,7 +89,7 @@ public ConsumeService(Map props, String name, CompletableFuture< Properties consumerProps = new Properties(); // Assign default config. This has the lowest priority. - consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, FALSE); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "kmf-consumer"); consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "kmf-consumer-group-" + new Random().nextInt()); @@ -100,7 +101,7 @@ public ConsumeService(Map props, String name, CompletableFuture< // Assign config specified for ConsumeService. consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); - consumerProps.put("zookeeper.connect", zkConnect); + consumerProps.put(CommonServiceConfig.ZOOKEEPER_CONNECT_CONFIG, zkConnect); // Assign config specified for consumer. This has the highest priority. consumerProps.putAll(consumerPropsOverride); @@ -109,15 +110,15 @@ public ConsumeService(Map props, String name, CompletableFuture< props.forEach(consumerProps::putIfAbsent); } _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); - topicPartitionReady.thenRun(() -> { + topicPartitionResult.thenRun(() -> { MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); List reporters = new ArrayList<>(); reporters.add(new JmxReporter(JMX_PREFIX)); Metrics metrics = new Metrics(metricConfig, reporters, new SystemTime()); Map tags = new HashMap<>(); - tags.put("name", _name); + tags.put(TAGS_NAME, _name); _adminClient = AdminClient.create(props); - _sensors = new ConsumeMetrics(metrics, tags, topic, topicPartitionReady); + _sensors = new ConsumeMetrics(metrics, tags, topic, topicPartitionResult); _consumeThread = new Thread(() -> { try { consume(); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 95cef3cc..3a9abdfb 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -72,7 +72,7 @@ public class MultiClusterTopicManagementService implements Service { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); private static final String METRIC_GROUP_NAME = "topic-management-service"; - private final CompletableFuture _topicPartitionReady = new CompletableFuture<>(); + private final CompletableFuture _topicPartitionResult = new CompletableFuture<>(); private final AtomicBoolean _isRunning = new AtomicBoolean(false); private final String _serviceName; private final Map _topicManagementByCluster; @@ -80,7 +80,7 @@ public class MultiClusterTopicManagementService implements Service { private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; - private final CompletableFuture _topicManagementReady; + private final CompletableFuture _topicManagementResult; @SuppressWarnings("unchecked") @@ -93,18 +93,18 @@ public MultiClusterTopicManagementService(Map props, String serv _topicManagementByCluster = initializeTopicManagementHelper(propsByCluster, topic); _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); - _topicManagementReady = new CompletableFuture<>(); + _topicManagementResult = new CompletableFuture<>(); _executor = Executors.newSingleThreadScheduledExecutor( r -> new Thread(r, _serviceName + "-multi-cluster-topic-management-service")); - _topicPartitionReady.complete(null); + _topicPartitionResult.complete(null); } - public CompletableFuture topicManagementReady() { - return _topicManagementReady; + public CompletableFuture topicManagementResult() { + return _topicManagementResult; } - public CompletableFuture topicPartitionReady() { - return _topicPartitionReady; + public CompletableFuture topicPartitionResult() { + return _topicPartitionResult; } private Map initializeTopicManagementHelper(Map propsByCluster, String topic) throws Exception { @@ -180,7 +180,7 @@ public void run() { for (TopicManagementHelper helper : _topicManagementByCluster.values()) { helper.maybeAddPartitions(minPartitionNum); } - _topicManagementReady.complete(null); + _topicManagementResult.complete(null); for (Map.Entry entry : _topicManagementByCluster.entrySet()) { String clusterName = entry.getKey(); TopicManagementHelper helper = entry.getValue(); diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index b41d3c24..87b6b926 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -23,16 +23,16 @@ */ public class TopicManagementService implements Service { private final MultiClusterTopicManagementService _multiClusterTopicManagementService; - CompletableFuture _topicPartitionReady = new CompletableFuture<>(); + CompletableFuture _topicPartitionResult = new CompletableFuture<>(); public TopicManagementService(Map props, String serviceName) throws Exception { Map serviceProps = createMultiClusterTopicManagementServiceProps(props, serviceName); _multiClusterTopicManagementService = new MultiClusterTopicManagementService(serviceProps, serviceName); - _topicPartitionReady.complete(null); + _topicPartitionResult.complete(null); } - public CompletableFuture topicPartitionReady() { - return _topicPartitionReady; + public CompletableFuture topicPartitionResult() { + return _topicPartitionResult; } /** @@ -69,8 +69,8 @@ public synchronized void start() { _multiClusterTopicManagementService.start(); } - public CompletableFuture topicManagementReady() { - return _multiClusterTopicManagementService.topicManagementReady(); + public CompletableFuture topicManagementResult() { + return _multiClusterTopicManagementService.topicManagementResult(); } diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index af2cc1a0..ddbbe737 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -44,7 +44,7 @@ public class BasicEndToEndTest implements Test { public BasicEndToEndTest(Map props, String name) throws Exception { _name = name; _topicManagementService = new TopicManagementService(props, name); - CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionReady(); + CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionResult(); _produceService = new ProduceService(props, name); _consumeService = new ConsumeService(props, name, topicPartitionReady); } @@ -52,7 +52,7 @@ public BasicEndToEndTest(Map props, String name) throws Exceptio @Override public void start() { _topicManagementService.start(); - CompletableFuture completableFuture = _topicManagementService.topicManagementReady(); + CompletableFuture completableFuture = _topicManagementService.topicManagementResult(); completableFuture.thenRun(() -> { try { _produceService.start(); From 67b9906e810fd886a5205d682bbb009da4e8eb4c Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 20 Dec 2019 13:37:11 -0800 Subject: [PATCH 067/192] SSL or any non-plaintext security protocols config (#189) Secure Sockets Layer (SSL) or any non-plaintext security protocol AdminClient SSL or any non-plaintext security protocols config Signed-off-by: Andrew Choi --- README.md | 50 +++++++++++++++++++++++++++++--------------------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/README.md b/README.md index f4e56110..a1b5707d 100644 --- a/README.md +++ b/README.md @@ -40,33 +40,44 @@ Kafka Monitor supports Apache Kafka 0.8 to 2.0: ### Configuration Tips -- We advise advanced users to run Kafka Monitor with -`./bin/kafka-monitor-start.sh config/kafka-monitor.properties`. The default +
    +
  1. We advise advanced users to run Kafka Monitor with +./bin/kafka-monitor-start.sh config/kafka-monitor.properties. The default kafka-monitor.properties in the repo provides an simple example of how to monitor a single cluster. You probably need to change the value of -`zookeeper.connect` and `bootstrap.servers` to point to your cluster. - -- The full list of configs and their documentation can be found in the code of +zookeeper.connect and bootstrap.servers to point to your cluster. +
  2. +
    +
  3. The full list of configs and their documentation can be found in the code of Config class for respective service, e.g. ProduceServiceConfig.java and -ConsumeServiceConfig.java. - -- You can specify multiple SingleClusterMonitor in the kafka-monitor.properties to +ConsumeServiceConfig.java.
  4. +
    +
  5. You can specify multiple SingleClusterMonitor in the kafka-monitor.properties to monitor multiple Kafka clusters in one Kafka Monitor process. As another -advanced use-case, you can point ProduceService and ConsumeService to two -different Kafka clusters that are connected by MirrorMaker to monitor their -end-to-end latency. - -- Kafka Monitor by default will automatically create the monitor topic based on -the e.g. `topic-management.replicationFactor` and `topic-management.partitionsToBrokersRatio` +advanced use-case, you can point ProduceService and ConsumeService to two different Kafka clusters that are connected by MirrorMaker to monitor their end-to-end latency.
  6. +
    +
  7. Kafka Monitor by default will automatically create the monitor topic based on +the e.g. topic-management.replicationFactor and topic-management.partitionsToBrokersRatio specified in the config. replicationFactor is 1 by default and you probably want to change it to the same replication factor as used for your existing -topics. You can disable auto topic creation by setting `produce.topic.topicCreationEnabled` to false. - -- Kafka Monitor can automatically increase partition count of the monitor topic +topics. You can disable auto topic creation by setting produce.topic.topicCreationEnabled to false. +
  8. +
    +
  9. Kafka Monitor can automatically increase partition count of the monitor topic to ensure partition# >= broker#. It can also reassign partition and trigger preferred leader election to ensure that each broker acts as leader of at least one partition of the monitor topic. To use this feature, use either -EndToEndTest or TopicManagementService in the properties file. +EndToEndTest or TopicManagementService in the properties file.
  10. +
    +
  11. When using Secure Sockets Layer (SSL) or any non-plaintext security protocol for AdminClient, please configure the following entries in the single-cluster-monitor props, produce.producer.props, as well as consume.consumer.props. https://docs.confluent.io/current/installation/configuration/admin-configs.html +
      +
    1. ssl.key.password
    2. +
    3. ssl.keystore.location
    4. +
    5. ssl.keystore.password
    6. +
    7. ssl.truststore.location
    8. +
    9. ssl.truststore.password
    10. +
    +
### Build Kafka Monitor @@ -145,6 +156,3 @@ attribute-name of the JMX metric in the query above. - [Design Overview](https://github.com/linkedin/kafka-monitor/wiki/Design-Overview) - [Service and App Overview](https://github.com/linkedin/kafka-monitor/wiki) - [Future Work](https://github.com/linkedin/kafka-monitor/wiki/Future-Work) - - - From 2ced9da4aa6a3f1e7787e716d3d471f335d5ed07 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sat, 21 Dec 2019 14:31:13 -0800 Subject: [PATCH 068/192] Secure Socket Layer security protocol properties (#190) Secure Socket Layer security protocol properties Signed-off-by: Andrew Choi --- config/kafka-monitor.properties | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index e0b4efe3..7258ca5a 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -25,7 +25,7 @@ # }, # ... # } - +# # TestClassName can be canonical name or simple name of any class that implements # interface com.linkedin.kmf.services.Test. These classes should be under # package com.linkedin.kmf.tests. @@ -38,7 +38,9 @@ # or ServiceClassName. The key for the test/service in the json map is used as name to # identify the test/service in the log or JMX metrics, which is useful if multiple # test/service with the same class.name are run in the same Kafka Monitor process. - +# +# If using Secure Socket Layer for security protocol, SSL properties must be defined under +# produce.producer.props, consume.consumer.props, as well as single-cluster-monitor props { "single-cluster-monitor": { From ba71d8d02544643818f112de7a0b1cf9237813a5 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 22 Dec 2019 19:02:22 -0800 Subject: [PATCH 069/192] Placement of Kafka Monitor Logo and enlarge. (#191) * Placement of Kafka Monitor Logo and enlarge. Signed-off-by: Andrew Choi --- README.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index a1b5707d..d60135c0 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,8 @@ -# Kafka Monitor -drawing +

+ +

+# Kafka Monitor [![Build Status](https://travis-ci.org/linkedin/kafka-monitor.svg?branch=master)](https://travis-ci.org/linkedin/kafka-monitor) Kafka Monitor is a framework to implement and execute long-running kafka From ac36103afdc55fc27995702da1c4becef044444f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 23 Dec 2019 18:09:34 -0800 Subject: [PATCH 070/192] class variables (#193) Factor common variables in produceService into class variables. Signed-off-by: Andrew Choi andchoi@linkedin.com --- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 3 ++- src/main/java/com/linkedin/kmf/services/ProduceService.java | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index e1898af6..032331a9 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -67,6 +67,7 @@ public class ConsumeService implements Service { private final AtomicBoolean _running; private final int _latencySlaMs; private AdminClient _adminClient; + private static final long CONSUME_THREAD_SLEEP_MS = 100; public ConsumeService(Map props, String name, CompletableFuture topicPartitionResult) throws Exception { _name = name; @@ -144,7 +145,7 @@ record = _consumer.receive(); _sensors._consumeError.record(); LOG.warn(_name + "/ConsumeService failed to receive record", e); // Avoid busy while loop - Thread.sleep(100); + Thread.sleep(CONSUME_THREAD_SLEEP_MS); continue; } diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 3a3ec6dd..cf9e8bd1 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -61,7 +61,6 @@ public class ProduceService implements Service { ProduceServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG }; - private final String _name; private final ProduceMetrics _sensors; private KMBaseProducer _producer; @@ -86,6 +85,7 @@ public class ProduceService implements Service { private final int _latencyPercentileMaxMs; private final int _latencyPercentileGranularityMs; private final AdminClient _adminClient; + private static final String KEY_SERIALIZER_CLASS = "org.apache.kafka.common.serialization.StringSerializer"; public ProduceService(Map props, String name) throws Exception { _name = name; @@ -144,8 +144,8 @@ private void initializeProducer(Map props) throws Exception { producerProps.put(ProducerConfig.RETRIES_CONFIG, "3"); producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MAX_VALUE); producerProps.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KEY_SERIALIZER_CLASS); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KEY_SERIALIZER_CLASS); // Assign config specified for ProduceService. producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, _producerId); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, _brokerList); From 39150ea43327d3aaf5fb8903056c0d95c286ece0 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 24 Dec 2019 12:45:25 -0800 Subject: [PATCH 071/192] =?UTF-8?q?checkstyle=20conventions=20and=20reduna?= =?UTF-8?q?nt=20uages=20of=20"this"=20=F0=9F=95=8D=20(#194)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit checkstyle conventions and redunant uages of .this Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/services/ProduceService.java | 10 +++++----- src/test/java/com/linkedin/kmf/KafkaMonitorTest.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index cf9e8bd1..53da7ddd 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -233,7 +233,7 @@ public boolean isRunning() { } private class ProduceMetrics { - public final Metrics metrics; + public final Metrics _metrics; private final Sensor _recordsProduced; private final Sensor _produceError; private final Sensor _produceDelay; @@ -243,8 +243,8 @@ private class ProduceMetrics { private final Map _tags; public ProduceMetrics(final Metrics metrics, final Map tags) { - this.metrics = metrics; - this._tags = tags; + _metrics = metrics; + _tags = tags; _recordsProducedPerPartition = new ConcurrentHashMap<>(); _produceErrorPerPartition = new ConcurrentHashMap<>(); @@ -307,12 +307,12 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { } void addPartitionSensors(int partition) { - Sensor recordsProducedSensor = metrics.sensor("records-produced-partition-" + partition); + Sensor recordsProducedSensor = _metrics.sensor("records-produced-partition-" + partition); recordsProducedSensor.add(new MetricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, "The average number of records per second that are produced to this partition", _tags), new Rate()); _recordsProducedPerPartition.put(partition, recordsProducedSensor); - Sensor errorsSensor = metrics.sensor("produce-error-partition-" + partition); + Sensor errorsSensor = _metrics.sensor("produce-error-partition-" + partition); errorsSensor.add(new MetricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, "The average number of errors per second when producing to this partition", _tags), new Rate()); _produceErrorPerPartition.put(partition, errorsSensor); diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index daa9e5ab..486d1453 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -109,7 +109,7 @@ public void start() { public synchronized void stop() { _isRunning.compareAndSet(true, false); stopCount.incrementAndGet(); - this.notifyAll(); + notifyAll(); } @Override From 238f23b99a011b33e01bb0a40e55ba58a925ac91 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 17 Jan 2020 16:05:50 -0800 Subject: [PATCH 072/192] [Application Configuration] (#198) Update for [Application Configuration](https://github.com/linkedin/kafka-monitor/wiki/App-Configuration) --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index d60135c0..82507f92 100644 --- a/README.md +++ b/README.md @@ -5,7 +5,7 @@ # Kafka Monitor [![Build Status](https://travis-ci.org/linkedin/kafka-monitor.svg?branch=master)](https://travis-ci.org/linkedin/kafka-monitor) -Kafka Monitor is a framework to implement and execute long-running kafka +Kafka Monitor (KMF) is a framework to implement and execute long-running kafka system tests in a real cluster. It complements Kafka’s existing system tests by capturing potential bugs or regressions that are only likely to occur after prolonged period of time or with low probability. Moreover, it allows you to monitor Kafka @@ -158,3 +158,4 @@ attribute-name of the JMX metric in the query above. - [Design Overview](https://github.com/linkedin/kafka-monitor/wiki/Design-Overview) - [Service and App Overview](https://github.com/linkedin/kafka-monitor/wiki) - [Future Work](https://github.com/linkedin/kafka-monitor/wiki/Future-Work) +- [Application Configuration](https://github.com/linkedin/kafka-monitor/wiki/App-Configuration) From 5c4f832c42fb913de67745e503cf27250f613ea9 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 19 Jan 2020 16:48:59 -0800 Subject: [PATCH 073/192] Commenting on semantic-build-versioning.gradle file (#200) * semantic gradle file update * simpler one liner --- semantic-build-versioning.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/semantic-build-versioning.gradle b/semantic-build-versioning.gradle index e69de29b..bee379f7 100644 --- a/semantic-build-versioning.gradle +++ b/semantic-build-versioning.gradle @@ -0,0 +1,2 @@ + +/* This is used by vivin:gradle-semantic-build-versioning plugin to generate versioned jar files. */ From dedec57e6bab21661119d9df4f33f926ff195f07 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 22 Jan 2020 09:56:32 -0800 Subject: [PATCH 074/192] more metrics to the reporter service for multi cluster monitor (#201) Added metrics to the Reporter Service on multi cluster monitoring. Signed-off-by: Andrew Choi --- config/multi-cluster-monitor.properties | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/config/multi-cluster-monitor.properties b/config/multi-cluster-monitor.properties index 4c8ef947..41c5f975 100644 --- a/config/multi-cluster-monitor.properties +++ b/config/multi-cluster-monitor.properties @@ -76,7 +76,13 @@ "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", "kmf.services:type=produce-service,name=*:produce-error-rate", - "kmf.services:type=consume-service,name=*:consume-error-rate" + "kmf.services:type=consume-service,name=*:consume-error-rate", + "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", + "kmf.services:type=commit-availability-service,name=*:commit-latency-avg", + "kmf.services:type=commit-availability-service,name=*:commit-availability-avg", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", + "kmf.services:type=commit-availability-service,name=*:offsets-committed-total", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-total" ] }, From f9880a29eae32d9f2957daecb310a1f7b4192ba1 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 22 Jan 2020 10:03:29 -0800 Subject: [PATCH 075/192] newConsumer deprecated polling API update and singletonList (#199) NewConsumer deprecated polling API update and singletonList For the original poll(long timeouts), since 2.0, it's been deprecated and the doc indicates using poll(Duration), which does not block beyond the timeout awaiting partition assignment. https://cwiki.apache.org/confluence/display/KAFKA/KIP-266%3A+Fix+consumer+indefinite+blocking+behavior Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/consumer/NewConsumer.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java index 0d827606..d33b5bbb 100644 --- a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java @@ -9,11 +9,12 @@ */ package com.linkedin.kmf.consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import java.util.Arrays; +import java.time.Duration; +import java.util.Collections; import java.util.Iterator; import java.util.Properties; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; /* * Wrap around the new consumer from Apache Kafka and implement the #KMBaseConsumer interface @@ -25,13 +26,13 @@ public class NewConsumer implements KMBaseConsumer { public NewConsumer(String topic, Properties consumerProperties) { _consumer = new KafkaConsumer<>(consumerProperties); - _consumer.subscribe(Arrays.asList(topic)); + _consumer.subscribe(Collections.singletonList(topic)); } @Override public BaseConsumerRecord receive() { if (_recordIter == null || !_recordIter.hasNext()) - _recordIter = _consumer.poll(Long.MAX_VALUE).iterator(); + _recordIter = _consumer.poll(Duration.ofMillis(Long.MAX_VALUE)).iterator(); ConsumerRecord record = _recordIter.next(); return new BaseConsumerRecord(record.topic(), record.partition(), record.offset(), record.key(), record.value()); From 490df48aaf07daf9ef83e9cd4d07319806dd541d Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 23 Jan 2020 10:16:57 -0800 Subject: [PATCH 076/192] Migration to log4j-slf4j-impl:2.13.0 from Log4j12 and Jolokia JVM Bump up Dependency (#197) Migration to log4j-slf4j-impl:2.13.0 from Log4j12 and Jolokia JVM Bump up Dependency Build gradle update for Log4j2 and Slf4j-log4j12 Signed-off-by: Andrew Choi andchoi@linkedin.com --- bin/kmf-run-class.sh | 2 +- bin/windows/kmf-run-class.bat | 6 ++--- build.gradle | 4 ++-- config/log4j.properties | 29 ------------------------- config/log4j2.properties | 41 +++++++++++++++++++++++++++++++++++ docker/Dockerfile | 2 +- 6 files changed, 48 insertions(+), 36 deletions(-) delete mode 100644 config/log4j.properties create mode 100644 config/log4j2.properties diff --git a/bin/kmf-run-class.sh b/bin/kmf-run-class.sh index d694ee0b..cfaa5109 100755 --- a/bin/kmf-run-class.sh +++ b/bin/kmf-run-class.sh @@ -50,7 +50,7 @@ fi # Log4j settings if [ -z "$KAFKA_LOG4J_OPTS" ]; then - KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/log4j.properties" + KAFKA_LOG4J_OPTS="-Dlog4j.configurationFile=$base_dir/config/log4j2.properties" fi KAFKA_LOG4J_OPTS="-Dkafka.logs.dir=$LOG_DIR $KAFKA_LOG4J_OPTS" diff --git a/bin/windows/kmf-run-class.bat b/bin/windows/kmf-run-class.bat index 559a965d..65fb1cf9 100644 --- a/bin/windows/kmf-run-class.bat +++ b/bin/windows/kmf-run-class.bat @@ -60,7 +60,7 @@ IF ["%LOG_DIR%"] EQU [""] ( rem Log4j settings IF ["%KAFKA_LOG4J_OPTS%"] EQU [""] ( - set KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%BASE_DIR%\config\log4j.properties + set KAFKA_LOG4J_OPTS=-Dlog4j.configurationFile=%BASE_DIR%\config\log4j2.properties ) ELSE ( # create logs directory IF not exist %LOG_DIR% ( @@ -170,7 +170,7 @@ REM fi REM Launch mode REM if [ "x$DAEMON_MODE" = "xtrue" ]; then -REM nohup $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH REM $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null &REM -REM elseREM +REM nohup $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH REM $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null &REM +REM elseREM REM exec $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS REM $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" REM fi diff --git a/build.gradle b/build.gradle index 41467516..41957acb 100644 --- a/build.gradle +++ b/build.gradle @@ -31,13 +31,13 @@ allprojects { dependencies { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' - compile 'org.slf4j:slf4j-log4j12:1.7.6' + compile 'org.apache.logging.log4j:log4j-slf4j-impl:2.13.0' compile 'org.apache.avro:avro:1.4.0' compile 'org.testng:testng:6.8.8' compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' compile 'org.json:json:20140107' compile 'com.fasterxml.jackson.core:jackson-databind:2.7.1' - compile 'org.jolokia:jolokia-jvm:1.3.3' + compile 'org.jolokia:jolokia-jvm:1.6.2' compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' diff --git a/config/log4j.properties b/config/log4j.properties deleted file mode 100644 index 8d4b79a9..00000000 --- a/config/log4j.properties +++ /dev/null @@ -1,29 +0,0 @@ -# Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this -# file except in compliance with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on -# an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - -log4j.rootLogger=INFO, stdout - -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.appender.kafkaClientAppender=org.apache.log4j.DailyRollingFileAppender -log4j.appender.kafkaClientAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.kafkaClientAppender.File=${kafka.logs.dir}/kafka-client.log -log4j.appender.kafkaClientAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.kafkaClientAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.logger.com.linkedin.kmf.core.KafkaMonitor=INFO, stdout -log4j.additivity.com.linkedin.kmf.core.KafkaMonitor=false - -log4j.logger.org.apache.kafka=WARN, kafkaClientAppender -log4j.additivity.org.apache.kafka=false - -log4j.logger.kafka=WARN, kafkaClientAppender -log4j.additivity.kafka=false - diff --git a/config/log4j2.properties b/config/log4j2.properties new file mode 100644 index 00000000..d44a0b82 --- /dev/null +++ b/config/log4j2.properties @@ -0,0 +1,41 @@ +# Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this +# file except in compliance with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on +# an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + +status = error +name = PropertiesConfig + +filters = threshold +filter.threshold.type = ThresholdFilter +filter.threshold.level = debug + +additivity.com.linkedin.kmf.core.KafkaMonitor = false +additivity.org.apache.kafka = false +additivity.kafka = false + +appenders = console + +appender.console.type = Console +appender.console.name = STDOUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + +appender.stdout=org.apache.log4j.ConsoleAppender +appender.stdout.layout=org.apache.log4j.PatternLayout +appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +appender.kafkaClientAppender=org.apache.log4j.DailyRollingFileAppender +appender.kafkaClientAppender.DatePattern='.'yyyy-MM-dd-HH +appender.kafkaClientAppender.File=${kafka.logs.dir}/kafka-client.log +appender.kafkaClientAppender.layout=org.apache.log4j.PatternLayout +appender.kafkaClientAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +rootLogger = INFO +rootLogger.level = info +rootLogger.appenderRefs = stdout +rootLogger.appenderRef.stdout.ref = STDOUT + diff --git a/docker/Dockerfile b/docker/Dockerfile index c47fbffb..8c6cbbd7 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -20,7 +20,7 @@ ADD build/ build/ ADD bin/kafka-monitor-start.sh bin/kafka-monitor-start.sh ADD bin/kmf-run-class.sh bin/kmf-run-class.sh ADD config/kafka-monitor.properties config/kafka-monitor.properties -ADD config/log4j.properties config/log4j.properties +ADD config/log4j2.properties config/log4j2.properties ADD docker/kafka-monitor-docker-entry.sh kafka-monitor-docker-entry.sh ADD webapp/ webapp/ From 327a5ac9cfb80333c2e9f3812ec4d2014de4e422 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 24 Jan 2020 08:42:09 -0800 Subject: [PATCH 077/192] commit availability update on README markdown (#202) README markdown Update for consumer offsets Commit availability Signed-off-by: Andrew Choi --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 82507f92..00bb6742 100644 --- a/README.md +++ b/README.md @@ -10,7 +10,8 @@ system tests in a real cluster. It complements Kafka’s existing system tests by capturing potential bugs or regressions that are only likely to occur after prolonged period of time or with low probability. Moreover, it allows you to monitor Kafka cluster using end-to-end pipelines to obtain a number of derived vital stats -such as end-to-end latency, service availability and message loss rate. You can easily +such as end-to-end latency, service availability, consumer offset commit availability, +as well as message loss rate. You can easily deploy Kafka Monitor to test and monitor your Kafka cluster without requiring any change to your application. From 4bf48036f88a17b3a7c83492d24de9886e1304fe Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 24 Jan 2020 11:36:48 -0800 Subject: [PATCH 078/192] 2020 Copyright linkedin updates (#203) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Copyright Updates for 2020. Happy New Year. 👷🏻‍♂️👷🏻‍♂️👷🏻‍♂️👷🏻‍♂️👷🏻‍♂️👷🏻‍♂️ Signed-off-by: Andrew Choi --- checkstyle/checkstyle.xml | 2 +- src/main/java/com/linkedin/kmf/KafkaMonitor.java | 2 +- src/main/java/com/linkedin/kmf/apps/App.java | 3 ++- src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java | 3 ++- src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 3 ++- .../linkedin/kmf/apps/configs/MultiClusterMonitorConfig.java | 3 ++- .../com/linkedin/kmf/common/ConfigDocumentationGenerator.java | 3 ++- src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java | 3 ++- .../java/com/linkedin/kmf/common/MbeanAttributeValue.java | 3 ++- src/main/java/com/linkedin/kmf/common/Utils.java | 3 ++- .../java/com/linkedin/kmf/consumer/BaseConsumerRecord.java | 3 ++- src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java | 3 ++- src/main/java/com/linkedin/kmf/consumer/NewConsumer.java | 3 ++- src/main/java/com/linkedin/kmf/partitioner/KMPartitioner.java | 3 ++- .../java/com/linkedin/kmf/partitioner/NewKMPartitioner.java | 2 +- .../java/com/linkedin/kmf/producer/BaseProducerRecord.java | 3 ++- src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java | 3 ++- src/main/java/com/linkedin/kmf/producer/NewProducer.java | 3 ++- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 2 +- .../linkedin/kmf/services/DefaultMetricsReporterService.java | 3 ++- .../linkedin/kmf/services/GraphiteMetricsReporterService.java | 4 ++-- src/main/java/com/linkedin/kmf/services/JettyService.java | 3 ++- src/main/java/com/linkedin/kmf/services/JolokiaService.java | 3 ++- .../linkedin/kmf/services/KafkaMetricsReporterService.java | 2 +- .../kmf/services/MultiClusterTopicManagementService.java | 2 +- src/main/java/com/linkedin/kmf/services/ProduceService.java | 2 +- src/main/java/com/linkedin/kmf/services/Service.java | 3 ++- .../linkedin/kmf/services/SignalFxMetricsReporterService.java | 1 + .../linkedin/kmf/services/StatsdMetricsReporterService.java | 4 ++-- .../com/linkedin/kmf/services/TopicManagementService.java | 2 +- .../linkedin/kmf/services/configs/CommonServiceConfig.java | 3 ++- .../linkedin/kmf/services/configs/ConsumeServiceConfig.java | 2 +- .../services/configs/DefaultMetricsReporterServiceConfig.java | 3 ++- .../configs/GraphiteMetricsReporterServiceConfig.java | 3 ++- .../com/linkedin/kmf/services/configs/JettyServiceConfig.java | 3 ++- .../services/configs/KafkaMetricsReporterServiceConfig.java | 3 ++- .../configs/MultiClusterTopicManagementServiceConfig.java | 3 ++- .../linkedin/kmf/services/configs/ProduceServiceConfig.java | 3 ++- .../configs/SignalFxMetricsReporterServiceConfig.java | 1 + .../services/configs/StatsdMetricsReporterServiceConfig.java | 3 ++- .../kmf/services/configs/TopicManagementServiceConfig.java | 3 ++- src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java | 3 ++- src/main/java/com/linkedin/kmf/tests/Test.java | 2 +- .../com/linkedin/kmf/topicfactory/DefaultTopicFactory.java | 3 ++- src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java | 3 ++- src/test/java/com/linkedin/kmf/KafkaMonitorTest.java | 3 ++- .../com/linkedin/kmf/services/TopicManagementServiceTest.java | 3 ++- 47 files changed, 82 insertions(+), 47 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 63736f49..da75cff1 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -18,7 +18,7 @@ - + diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index a158407c..cd2dddfc 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/apps/App.java b/src/main/java/com/linkedin/kmf/apps/App.java index c1ac8ee0..c0b9071d 100644 --- a/src/main/java/com/linkedin/kmf/apps/App.java +++ b/src/main/java/com/linkedin/kmf/apps/App.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.apps; public interface App { diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index 04cbb650..a5e48de3 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.apps; import com.linkedin.kmf.apps.configs.MultiClusterMonitorConfig; diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 3ce1dda2..c8a16df0 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.apps; import com.linkedin.kmf.services.ConsumeService; diff --git a/src/main/java/com/linkedin/kmf/apps/configs/MultiClusterMonitorConfig.java b/src/main/java/com/linkedin/kmf/apps/configs/MultiClusterMonitorConfig.java index 9fe128c7..d99d0d39 100644 --- a/src/main/java/com/linkedin/kmf/apps/configs/MultiClusterMonitorConfig.java +++ b/src/main/java/com/linkedin/kmf/apps/configs/MultiClusterMonitorConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -48,4 +48,5 @@ public Double getDouble(String key) { public MultiClusterMonitorConfig(Map props) { super(CONFIG, props); } + } diff --git a/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java index 70011ffc..b6a1dcdb 100644 --- a/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java +++ b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.common; import java.io.File; diff --git a/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java b/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java index a18053be..f5ad9291 100644 --- a/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java +++ b/src/main/java/com/linkedin/kmf/common/DefaultTopicSchema.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.common; import java.util.Arrays; diff --git a/src/main/java/com/linkedin/kmf/common/MbeanAttributeValue.java b/src/main/java/com/linkedin/kmf/common/MbeanAttributeValue.java index 3dff36cf..0e0d3d9b 100644 --- a/src/main/java/com/linkedin/kmf/common/MbeanAttributeValue.java +++ b/src/main/java/com/linkedin/kmf/common/MbeanAttributeValue.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.common; public class MbeanAttributeValue { diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 4909d2d0..bf35a0eb 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.common; import java.io.ByteArrayOutputStream; diff --git a/src/main/java/com/linkedin/kmf/consumer/BaseConsumerRecord.java b/src/main/java/com/linkedin/kmf/consumer/BaseConsumerRecord.java index 17ce0c75..89447d7e 100644 --- a/src/main/java/com/linkedin/kmf/consumer/BaseConsumerRecord.java +++ b/src/main/java/com/linkedin/kmf/consumer/BaseConsumerRecord.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.consumer; public class BaseConsumerRecord { diff --git a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java index 6b3ba13f..fba9dbd8 100644 --- a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.consumer; /** diff --git a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java index d33b5bbb..f9515067 100644 --- a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.consumer; import java.time.Duration; diff --git a/src/main/java/com/linkedin/kmf/partitioner/KMPartitioner.java b/src/main/java/com/linkedin/kmf/partitioner/KMPartitioner.java index 839e3d77..d35d975d 100644 --- a/src/main/java/com/linkedin/kmf/partitioner/KMPartitioner.java +++ b/src/main/java/com/linkedin/kmf/partitioner/KMPartitioner.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.partitioner; public interface KMPartitioner { diff --git a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java index 348eb96c..62fce556 100644 --- a/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java +++ b/src/main/java/com/linkedin/kmf/partitioner/NewKMPartitioner.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/producer/BaseProducerRecord.java b/src/main/java/com/linkedin/kmf/producer/BaseProducerRecord.java index 22548383..cc5a5c44 100644 --- a/src/main/java/com/linkedin/kmf/producer/BaseProducerRecord.java +++ b/src/main/java/com/linkedin/kmf/producer/BaseProducerRecord.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.producer; public class BaseProducerRecord { diff --git a/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java b/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java index 2ebc6ca0..56132127 100644 --- a/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java +++ b/src/main/java/com/linkedin/kmf/producer/KMBaseProducer.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.producer; import org.apache.kafka.clients.producer.RecordMetadata; diff --git a/src/main/java/com/linkedin/kmf/producer/NewProducer.java b/src/main/java/com/linkedin/kmf/producer/NewProducer.java index 7379dc7b..80a6a9ca 100644 --- a/src/main/java/com/linkedin/kmf/producer/NewProducer.java +++ b/src/main/java/com/linkedin/kmf/producer/NewProducer.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.producer; import org.apache.kafka.clients.producer.KafkaProducer; diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 032331a9..ea3e77b8 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java index 8345209d..ac47473f 100644 --- a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services; import com.linkedin.kmf.common.MbeanAttributeValue; diff --git a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java index a9025e1f..00c3d295 100644 --- a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,8 +7,8 @@ * 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. */ -package com.linkedin.kmf.services; +package com.linkedin.kmf.services; import com.linkedin.kmf.common.MbeanAttributeValue; import com.linkedin.kmf.services.configs.GraphiteMetricsReporterServiceConfig; diff --git a/src/main/java/com/linkedin/kmf/services/JettyService.java b/src/main/java/com/linkedin/kmf/services/JettyService.java index e8cc1f25..9cd30c75 100644 --- a/src/main/java/com/linkedin/kmf/services/JettyService.java +++ b/src/main/java/com/linkedin/kmf/services/JettyService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services; import com.linkedin.kmf.services.configs.JettyServiceConfig; diff --git a/src/main/java/com/linkedin/kmf/services/JolokiaService.java b/src/main/java/com/linkedin/kmf/services/JolokiaService.java index 360ac2ab..053e9108 100644 --- a/src/main/java/com/linkedin/kmf/services/JolokiaService.java +++ b/src/main/java/com/linkedin/kmf/services/JolokiaService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services; import java.util.Map; diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index c861823c..a1deb1cb 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at *

* http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 3a9abdfb..db4ca4a7 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 53da7ddd..ada601e3 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/Service.java b/src/main/java/com/linkedin/kmf/services/Service.java index d18318b3..fb343b58 100644 --- a/src/main/java/com/linkedin/kmf/services/Service.java +++ b/src/main/java/com/linkedin/kmf/services/Service.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services; /** diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java index 389d4b53..78b3733d 100644 --- a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -1,6 +1,7 @@ /* * Copyright (C) 2018 SignalFx, Inc. Licensed under the Apache 2 License. */ + package com.linkedin.kmf.services; import com.codahale.metrics.MetricRegistry; diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java index a9603807..f59b7631 100644 --- a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,8 +7,8 @@ * 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. */ -package com.linkedin.kmf.services; +package com.linkedin.kmf.services; import com.linkedin.kmf.common.MbeanAttributeValue; import com.linkedin.kmf.services.configs.StatsdMetricsReporterServiceConfig; diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index 87b6b926..a39e0677 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java index 46316c01..af32c169 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/CommonServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java index a15ee61c..bd310a2a 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ConsumeServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java index 0c8c9130..26393d87 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java index 827cb7e4..f78a04c6 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/com/linkedin/kmf/services/configs/JettyServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/JettyServiceConfig.java index 38676631..dc6f8850 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/JettyServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/JettyServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java index d1f0b374..c97c6f84 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java index 337f86f8..ddb40394 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/MultiClusterTopicManagementServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import java.util.Map; diff --git a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java index fcf8bb13..66988066 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/ProduceServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import com.linkedin.kmf.partitioner.NewKMPartitioner; diff --git a/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java index 55637e9a..8781a969 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java @@ -1,6 +1,7 @@ /* * Copyright (C) 2018 SignalFx, Inc. Licensed under the Apache 2 License. */ + package com.linkedin.kmf.services.configs; import java.util.Arrays; diff --git a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java index 9ed2132e..60639931 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -11,6 +11,7 @@ * In order to enable the StatsD metrics export, add the following section to kafka-monitor.properties file * */ + package com.linkedin.kmf.services.configs; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java index 9f79b30f..cf606c35 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/TopicManagementServiceConfig.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services.configs; import com.linkedin.kmf.topicfactory.DefaultTopicFactory; diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index ddbbe737..3c2e0724 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.tests; import com.linkedin.kmf.apps.SingleClusterMonitor; diff --git a/src/main/java/com/linkedin/kmf/tests/Test.java b/src/main/java/com/linkedin/kmf/tests/Test.java index 09e6a10a..1ce6317c 100644 --- a/src/main/java/com/linkedin/kmf/tests/Test.java +++ b/src/main/java/com/linkedin/kmf/tests/Test.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java index 96803a80..96c6a0bd 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/DefaultTopicFactory.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.topicfactory; import com.linkedin.kmf.common.Utils; diff --git a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java index 45b4da7c..a0e5b46e 100644 --- a/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java +++ b/src/main/java/com/linkedin/kmf/topicfactory/TopicFactory.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.topicfactory; import java.util.Properties; diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index 486d1453..7a4de10e 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf; import com.linkedin.kmf.services.Service; diff --git a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java index 53ae7807..23bf1d8b 100644 --- a/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/TopicManagementServiceTest.java @@ -1,5 +1,5 @@ /** - * Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this * file except in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 @@ -7,6 +7,7 @@ * 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. */ + package com.linkedin.kmf.services; import com.linkedin.kmf.services.MultiClusterTopicManagementService.TopicManagementHelper; From cbba9a95a57330e7e503bebfcbb95ec38f37731f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 30 Jan 2020 12:36:15 -0800 Subject: [PATCH 079/192] Commit Availability Service (#196) **Commit Availability Service.** 1 - Addresses: Broken ConsumeService on latest release #195 https://github.com/linkedin/kafka-monitor/issues/195 2 - Addresses: https://github.com/linkedin/kafka-monitor/issues/204 3 - Performs Consumer Offset Commit Availability using Metric Sensors. Does periodically, every few x seconds. 4 - In doing this, public interface `ConsumerFactory` is created to build consumer and pass into `consumeService`. WIP as a follow up PR : Commit `Latency` Service Signed-off-by: Andrew Choi --- build.gradle | 1 + config/kafka-monitor.properties | 56 +++- .../java/com/linkedin/kmf/KafkaMonitor.java | 35 ++- .../kmf/apps/MultiClusterMonitor.java | 4 +- .../kmf/apps/SingleClusterMonitor.java | 29 +- .../common/ConfigDocumentationGenerator.java | 3 + .../linkedin/kmf/consumer/KMBaseConsumer.java | 18 ++ .../linkedin/kmf/consumer/NewConsumer.java | 38 +++ .../services/CommitAvailabilityMetrics.java | 55 ++++ .../linkedin/kmf/services/ConsumeMetrics.java | 120 +++++++++ .../linkedin/kmf/services/ConsumeService.java | 247 ++++++------------ .../kmf/services/ConsumerFactory.java | 32 +++ .../kmf/services/ConsumerFactoryImpl.java | 113 ++++++++ .../DefaultMetricsReporterService.java | 2 + .../services/KafkaMetricsReporterService.java | 1 + .../MultiClusterTopicManagementService.java | 2 + .../linkedin/kmf/services/ProduceService.java | 1 + .../com/linkedin/kmf/services/Service.java | 5 + .../SignalFxMetricsReporterService.java | 1 + .../StatsdMetricsReporterService.java | 1 + .../kmf/services/TopicManagementService.java | 1 + .../linkedin/kmf/tests/BasicEndToEndTest.java | 4 +- .../com/linkedin/kmf/KafkaMonitorTest.java | 2 +- .../kmf/services/ConsumeServiceTest.java | 187 +++++++++++++ 24 files changed, 762 insertions(+), 196 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java create mode 100644 src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java create mode 100644 src/main/java/com/linkedin/kmf/services/ConsumerFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java create mode 100644 src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java diff --git a/build.gradle b/build.gradle index 41957acb..1f672cb8 100644 --- a/build.gradle +++ b/build.gradle @@ -43,6 +43,7 @@ allprojects { compile 'com.signalfx.public:signalfx-codahale:0.0.47' compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.3.1' + testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' } diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 7258ca5a..97c3ee29 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -38,8 +38,8 @@ # or ServiceClassName. The key for the test/service in the json map is used as name to # identify the test/service in the log or JMX metrics, which is useful if multiple # test/service with the same class.name are run in the same Kafka Monitor process. -# -# If using Secure Socket Layer for security protocol, SSL properties must be defined under +# +# If using Secure Socket Layer for security protocol, SSL properties must be defined under # produce.producer.props, consume.consumer.props, as well as single-cluster-monitor props { @@ -82,21 +82,49 @@ "class.name": "com.linkedin.kmf.services.DefaultMetricsReporterService", "report.interval.sec": 1, "report.metrics.list": [ - "kmf:type=kafka-monitor:offline-runnable-count", - "kmf.services:type=produce-service,name=*:produce-availability-avg", - "kmf.services:type=consume-service,name=*:consume-availability-avg", - "kmf.services:type=produce-service,name=*:records-produced-total", - "kmf.services:type=consume-service,name=*:records-consumed-total", - "kmf.services:type=consume-service,name=*:records-lost-total", - "kmf.services:type=consume-service,name=*:records-lost-rate", - "kmf.services:type=consume-service,name=*:records-duplicated-total", - "kmf.services:type=consume-service,name=*:records-delay-ms-avg", - "kmf.services:type=produce-service,name=*:records-produced-rate", - "kmf.services:type=produce-service,name=*:produce-error-rate", - "kmf.services:type=consume-service,name=*:consume-error-rate" + "kmf:type=kafka-monitor,name=*:offline-runnable-count", + "kmf.services:type=produce-service,name=*:produce-availability-avg", + "kmf.services:type=consume-service,name=*:consume-availability-avg", + "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", + "kmf.services:type=commit-availability-service,name=*:commit-latency-avg", + "kmf.services:type=commit-availability-service,name=*:commit-availability-avg", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", + "kmf.services:type=produce-service,name=*:records-produced-total", + "kmf.services:type=consume-service,name=*:records-consumed-total", + "kmf.services:type=consume-service,name=*:records-lost-total", + "kmf.services:type=consume-service,name=*:records-lost-rate", + "kmf.services:type=consume-service,name=*:records-duplicated-total", + "kmf.services:type=consume-service,name=*:records-delay-ms-avg", + "kmf.services:type=produce-service,name=*:records-produced-rate", + "kmf.services:type=produce-service,name=*:produce-error-rate", + "kmf.services:type=consume-service,name=*:consume-error-rate", + "kmf.services:type=commit-availability-service,name=*:offsets-committed-total", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-total" ] } +# Example produce-service to produce messages to cluster +# "produce-service": { +# "class.name": "com.linkedin.kmf.services.ProduceService", +# "topic": "kafka-monitor-topic", +# "zookeeper.connect": "localhost:2181", +# "bootstrap.servers": "localhost:9092", +# "consume.latency.sla.ms": "20000", +# "consume.consumer.props": { +# } +# }, + +# Example consume-service to consume messages +# "consume-service": { +# "class.name": "com.linkedin.kmf.services.ConsumeService", +# "topic": "kafka-monitor-topic", +# "zookeeper.connect": "localhost:2181", +# "bootstrap.servers": "localhost:9092", +# "consume.latency.sla.ms": "20000", +# "consume.consumer.props": { +# } +# }, + # Example statsd-service to report metrics # "statsd-service": { # "class.name": "com.linkedin.kmf.services.StatsdMetricsReporterService", diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index cd2dddfc..e8d48a8a 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -11,12 +11,15 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.kmf.apps.App; +import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.Service; import java.io.BufferedReader; import java.io.FileReader; +import java.lang.reflect.Constructor; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; @@ -31,7 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * This is the main entry point of the monitor. It reads the configuration and manages the life cycle of the monitoring * applications. @@ -63,13 +65,24 @@ public KafkaMonitor(Map testProps) throws Exception { throw new IllegalArgumentException(name + " is not configured with " + CLASS_NAME_CONFIG); String className = (String) props.get(CLASS_NAME_CONFIG); - Class cls = Class.forName(className); - if (App.class.isAssignableFrom(cls)) { + Class aClass = Class.forName(className); + if (App.class.isAssignableFrom(aClass)) { App test = (App) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); _apps.put(name, test); - } else if (Service.class.isAssignableFrom(cls)) { - Service service = (Service) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); - _services.put(name, service); + } else if (Service.class.isAssignableFrom(aClass)) { + Constructor[] constructors = Class.forName(className).getConstructors(); + if (this.constructorContainsFuture(constructors)) { + CompletableFuture completableFuture = new CompletableFuture<>(); + completableFuture.complete(null); + ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); + Service service = (Service) Class.forName(className) + .getConstructor(String.class, CompletableFuture.class, ConsumerFactoryImpl.class) + .newInstance(name, completableFuture, consumerFactory); + _services.put(name, service); + } else { + Service service = (Service) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); + _services.put(name, service); + } } else { throw new IllegalArgumentException(className + " should implement either " + App.class.getSimpleName() + " or " + Service.class.getSimpleName()); } @@ -83,6 +96,15 @@ public KafkaMonitor(Map testProps) throws Exception { (config, now) -> _offlineRunnables.size()); } + private boolean constructorContainsFuture(Constructor[] constructors) { + for (int n = 0; n < constructors[0].getParameterTypes().length; ++n) { + if (constructors[0].getParameterTypes()[n].equals(CompletableFuture.class)) { + return true; + } + } + return false; + } + public synchronized void start() { if (!_isRunning.compareAndSet(false, true)) { return; @@ -151,7 +173,6 @@ public static void main(String[] args) throws Exception { return; } - StringBuilder buffer = new StringBuilder(); try (BufferedReader br = new BufferedReader(new FileReader(args[0].trim()))) { String line; diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index a5e48de3..b1fcb0c2 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -12,6 +12,7 @@ import com.linkedin.kmf.apps.configs.MultiClusterMonitorConfig; import com.linkedin.kmf.services.ConsumeService; +import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.MultiClusterTopicManagementService; import com.linkedin.kmf.services.ProduceService; import java.util.HashMap; @@ -42,7 +43,8 @@ public MultiClusterMonitor(Map props, String name) throws Except _multiClusterTopicManagementService = new MultiClusterTopicManagementService(createMultiClusterTopicManagementServiceProps(props, config), name); CompletableFuture topicPartitionReady = _multiClusterTopicManagementService.topicPartitionResult(); _produceService = new ProduceService(createProduceServiceProps(props, config), name); - _consumeService = new ConsumeService(createConsumeServiceProps(props, config), name, topicPartitionReady); + ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(createConsumeServiceProps(props, config)); + _consumeService = new ConsumeService(name, topicPartitionReady, consumerFactory); } @SuppressWarnings("unchecked") diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index c8a16df0..ec7ba843 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -11,16 +11,19 @@ package com.linkedin.kmf.apps; import com.linkedin.kmf.services.ConsumeService; +import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.DefaultMetricsReporterService; import com.linkedin.kmf.services.JettyService; import com.linkedin.kmf.services.JolokiaService; import com.linkedin.kmf.services.ProduceService; +import com.linkedin.kmf.services.Service; import com.linkedin.kmf.services.TopicManagementService; import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import com.linkedin.kmf.services.configs.DefaultMetricsReporterServiceConfig; import com.linkedin.kmf.services.configs.MultiClusterTopicManagementServiceConfig; import com.linkedin.kmf.services.configs.ProduceServiceConfig; import com.linkedin.kmf.services.configs.TopicManagementServiceConfig; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -49,14 +52,20 @@ public class SingleClusterMonitor implements App { private final ProduceService _produceService; private final ConsumeService _consumeService; private final String _name; + private final List _allServices; public SingleClusterMonitor(Map props, String name) throws Exception { _name = name; _topicManagementService = new TopicManagementService(props, name); - CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionResult(); _produceService = new ProduceService(props, name); - _consumeService = new ConsumeService(props, name, topicPartitionReady); + ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); + _consumeService = new ConsumeService(name, topicPartitionReady, consumerFactory); + int servicesInitialCapacity = 4; + _allServices = new ArrayList<>(servicesInitialCapacity); + _allServices.add(_topicManagementService); + _allServices.add(_produceService); + _allServices.add(_consumeService); } @Override @@ -72,9 +81,9 @@ public void start() { @Override public void stop() { - _topicManagementService.stop(); - _produceService.stop(); - _consumeService.stop(); + for (Service service : _allServices) { + service.stop(); + } LOG.info(_name + "/SingleClusterMonitor stopped."); } @@ -99,9 +108,9 @@ public boolean isRunning() { @Override public void awaitShutdown() { - _topicManagementService.awaitShutdown(); - _produceService.awaitShutdown(); - _consumeService.awaitShutdown(); + for (Service service : _allServices) { + service.awaitShutdown(); + } } /** Get the command-line argument parser. */ @@ -305,7 +314,9 @@ public static void main(String[] args) throws Exception { "kmf.services:type=consume-service,name=*:records-delay-ms-avg", "kmf.services:type=produce-service,name=*:records-produced-rate", "kmf.services:type=produce-service,name=*:produce-error-rate", - "kmf.services:type=consume-service,name=*:consume-error-rate" + "kmf.services:type=consume-service,name=*:consume-error-rate", + "kmf.services:type=consume-service,name=*:commit-latency-avg", + "kmf.services:type=consume-service,name=*:commit-availability-avg" ); props.put(DefaultMetricsReporterServiceConfig.REPORT_METRICS_CONFIG, metrics); diff --git a/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java index b6a1dcdb..49247292 100644 --- a/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java +++ b/src/main/java/com/linkedin/kmf/common/ConfigDocumentationGenerator.java @@ -17,12 +17,15 @@ import java.lang.reflect.Field; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Generates the table of configuration parameters, their documentation strings and default values. */ public class ConfigDocumentationGenerator { + private static final Logger LOG = LoggerFactory.getLogger(ConfigDocumentationGenerator.class); private static void printHelp() { System.out.println("ConfigDocumentationGenerator outputDirectory configClassNames..."); diff --git a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java index fba9dbd8..7cb99671 100644 --- a/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/KMBaseConsumer.java @@ -10,6 +10,12 @@ package com.linkedin.kmf.consumer; +import java.util.Map; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; + + /** * A base consumer used to abstract different consumer classes. * @@ -20,6 +26,18 @@ public interface KMBaseConsumer { BaseConsumerRecord receive() throws Exception; + void commitAsync(); + + void commitAsync(final Map offsets, OffsetCommitCallback callback); + + void commitAsync(OffsetCommitCallback callback); + + OffsetAndMetadata committed(TopicPartition tp); + void close(); + long lastCommitted(); + + void updateLastCommit(); + } diff --git a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java index f9515067..fac8fcf0 100644 --- a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java @@ -13,9 +13,15 @@ import java.time.Duration; import java.util.Collections; import java.util.Iterator; +import java.util.Map; import java.util.Properties; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /* * Wrap around the new consumer from Apache Kafka and implement the #KMBaseConsumer interface @@ -24,6 +30,8 @@ public class NewConsumer implements KMBaseConsumer { private final KafkaConsumer _consumer; private Iterator> _recordIter; + private static final Logger LOG = LoggerFactory.getLogger(NewConsumer.class); + private static long lastCommitted; public NewConsumer(String topic, Properties consumerProperties) { _consumer = new KafkaConsumer<>(consumerProperties); @@ -39,9 +47,39 @@ public BaseConsumerRecord receive() { return new BaseConsumerRecord(record.topic(), record.partition(), record.offset(), record.key(), record.value()); } + @Override + public void commitAsync() { + _consumer.commitAsync(); + } + + @Override + public void commitAsync(final Map offsets, OffsetCommitCallback callback) { + _consumer.commitAsync(offsets, callback); + } + + @Override + public void commitAsync(OffsetCommitCallback callback) { + _consumer.commitAsync(callback); + } + + @Override + public OffsetAndMetadata committed(TopicPartition tp) { + return _consumer.committed(tp); + } + @Override public void close() { _consumer.close(); } + @Override + public long lastCommitted() { + return lastCommitted; + } + + @Override + public void updateLastCommit() { + lastCommitted = System.currentTimeMillis(); + } + } diff --git a/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java b/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java new file mode 100644 index 00000000..30a2a2d4 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java @@ -0,0 +1,55 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.metrics.stats.Total; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +class CommitAvailabilityMetrics { + + private static final String METRIC_GROUP_NAME = "commit-availability-service"; + private static final Logger LOG = LoggerFactory.getLogger(CommitAvailabilityMetrics.class); + public final Sensor _offsetsCommitted; + public final Sensor _failedCommitOffsets; + + /** + * Metrics for Calculating the offset commit availability of a consumer. + * @param metrics the commit offset metrics + * @param tags the tags associated, i.e) kmf.services:name=single-cluster-monitor + */ + CommitAvailabilityMetrics(final Metrics metrics, final Map tags) { + LOG.info("{} called.", this.getClass().getSimpleName()); + _offsetsCommitted = metrics.sensor("offsets-committed"); + _offsetsCommitted.add(new MetricName("offsets-committed-total", METRIC_GROUP_NAME, + "The total number of offsets per second that are committed.", tags), new Total()); + + _failedCommitOffsets = metrics.sensor("failed-commit-offsets"); + _failedCommitOffsets.add(new MetricName("failed-commit-offsets-avg", METRIC_GROUP_NAME, + "The average number of offsets per second that have failed.", tags), new Rate()); + _failedCommitOffsets.add(new MetricName("failed-commit-offsets-total", METRIC_GROUP_NAME, + "The total number of offsets per second that have failed.", tags), new Total()); + + metrics.addMetric(new MetricName("offsets-committed-avg", METRIC_GROUP_NAME, "The average offset commits availability.", tags), + (MetricConfig config, long now) -> { + double offsetsCommittedCount = metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).value(); + double offsetsCommittedErrorCount = metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).value(); + return offsetsCommittedCount / (offsetsCommittedCount + offsetsCommittedErrorCount); + }); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java new file mode 100644 index 00000000..2a5c83aa --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java @@ -0,0 +1,120 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Percentile; +import org.apache.kafka.common.metrics.stats.Percentiles; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.metrics.stats.Total; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ConsumeMetrics { + final Sensor _consumeError; + final Sensor _bytesConsumed; + final Sensor _recordsConsumed; + final Sensor _recordsDuplicated; + final Sensor _recordsLost; + final Sensor _recordsDelay; + final Sensor _recordsDelayed; + private static final String METRIC_GROUP_NAME = "consume-service"; + private static final Logger LOG = LoggerFactory.getLogger(ConsumeMetrics.class); + + ConsumeMetrics(final Metrics metrics, + final Map tags, + String topicName, + CompletableFuture topicPartitionReady, + AdminClient adminClient, + final int latencyPercentileMaxMs, + final int latencyPercentileGranularityMs) { + topicPartitionReady.thenRun(() -> { + DescribeTopicsResult describeTopicsResult = adminClient.describeTopics(Collections.singleton(topicName)); + Map> topicResultValues = describeTopicsResult.values(); + KafkaFuture topicDescriptionKafkaFuture = topicResultValues.get(topicName); + TopicDescription topicDescription = null; + try { + topicDescription = topicDescriptionKafkaFuture.get(); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Exception occurred while retrieving the topic description.", e); + } + + int partitionCount = topicDescription.partitions().size(); + Sensor topicPartitionCount = metrics.sensor("topic-partitions"); + topicPartitionCount.add( + new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), + new Total(partitionCount)); + }); + + _bytesConsumed = metrics.sensor("bytes-consumed"); + _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); + + _consumeError = metrics.sensor("consume-error"); + _consumeError.add(new MetricName("consume-error-rate", METRIC_GROUP_NAME, "The average number of errors per second", tags), new Rate()); + _consumeError.add(new MetricName("consume-error-total", METRIC_GROUP_NAME, "The total number of errors", tags), new Total()); + + _recordsConsumed = metrics.sensor("records-consumed"); + _recordsConsumed.add(new MetricName("records-consumed-rate", METRIC_GROUP_NAME, "The average number of records per second that are consumed", tags), new Rate()); + _recordsConsumed.add(new MetricName("records-consumed-total", METRIC_GROUP_NAME, "The total number of records that are consumed", tags), new Total()); + + _recordsDuplicated = metrics.sensor("records-duplicated"); + _recordsDuplicated.add(new MetricName("records-duplicated-rate", METRIC_GROUP_NAME, "The average number of records per second that are duplicated", tags), new Rate()); + _recordsDuplicated.add(new MetricName("records-duplicated-total", METRIC_GROUP_NAME, "The total number of records that are duplicated", tags), new Total()); + + _recordsLost = metrics.sensor("records-lost"); + _recordsLost.add(new MetricName("records-lost-rate", METRIC_GROUP_NAME, "The average number of records per second that are lost", tags), new Rate()); + _recordsLost.add(new MetricName("records-lost-total", METRIC_GROUP_NAME, "The total number of records that are lost", tags), new Total()); + + _recordsDelayed = metrics.sensor("records-delayed"); + _recordsDelayed.add(new MetricName("records-delayed-rate", METRIC_GROUP_NAME, "The average number of records per second that are either lost or arrive after maximum allowed latency under SLA", tags), new Rate()); + _recordsDelayed.add(new MetricName("records-delayed-total", METRIC_GROUP_NAME, "The total number of records that are either lost or arrive after maximum allowed latency under SLA", tags), new Total()); + + _recordsDelay = metrics.sensor("records-delay"); + _recordsDelay.add(new MetricName("records-delay-ms-avg", METRIC_GROUP_NAME, "The average latency of records from producer to consumer", tags), new Avg()); + _recordsDelay.add(new MetricName("records-delay-ms-max", METRIC_GROUP_NAME, "The maximum latency of records from producer to consumer", tags), new Max()); + + // There are 2 extra buckets use for values smaller than 0.0 or larger than max, respectively. + int bucketNum = latencyPercentileMaxMs / latencyPercentileGranularityMs + 2; + int sizeInBytes = 4 * bucketNum; + _recordsDelay.add(new Percentiles(sizeInBytes, latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, + new Percentile(new MetricName("records-delay-ms-99th", METRIC_GROUP_NAME, "The 99th percentile latency of records from producer to consumer", tags), 99.0), + new Percentile(new MetricName("records-delay-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile latency of records from producer to consumer", tags), 99.9), + new Percentile(new MetricName("records-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of records from producer to consumer", tags), 99.99))); + + metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), + (config, now) -> { + double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); + + if (new Double(recordsLostRate).isNaN()) + recordsLostRate = 0; + if (new Double(recordsDelayedRate).isNaN()) + recordsDelayedRate = 0; + + return recordsConsumedRate + recordsLostRate > 0 + ? (recordsConsumedRate - recordsDelayedRate) / (recordsConsumedRate + recordsLostRate) : 0; + }); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index ea3e77b8..db20e6cd 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -7,132 +7,82 @@ * 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. */ + package com.linkedin.kmf.services; import com.linkedin.kmf.common.DefaultTopicSchema; import com.linkedin.kmf.common.Utils; import com.linkedin.kmf.consumer.BaseConsumerRecord; import com.linkedin.kmf.consumer.KMBaseConsumer; -import com.linkedin.kmf.consumer.NewConsumer; -import com.linkedin.kmf.services.configs.CommonServiceConfig; -import com.linkedin.kmf.services.configs.ConsumeServiceConfig; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; -import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.DescribeTopicsResult; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.metrics.stats.Max; -import org.apache.kafka.common.metrics.stats.Percentile; -import org.apache.kafka.common.metrics.stats.Percentiles; -import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.common.metrics.stats.Total; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class ConsumeService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ConsumeService.class); - private static final String METRIC_GROUP_NAME = "consume-service"; private static final String TAGS_NAME = "name"; - private static final String FALSE = "false"; - private static final String[] NON_OVERRIDABLE_PROPERTIES = - new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; - private final String _name; + private static final long COMMIT_TIME_INTERVAL = 4; + private static final long CONSUME_THREAD_SLEEP_MS = 100; + private static Metrics metrics; + private final AtomicBoolean _running; + private final KMBaseConsumer _baseConsumer; + private int _latencySlaMs; private ConsumeMetrics _sensors; - private final KMBaseConsumer _consumer; private Thread _consumeThread; - private final int _latencyPercentileMaxMs; - private final int _latencyPercentileGranularityMs; - private final AtomicBoolean _running; - private final int _latencySlaMs; private AdminClient _adminClient; - private static final long CONSUME_THREAD_SLEEP_MS = 100; - - public ConsumeService(Map props, String name, CompletableFuture topicPartitionResult) throws Exception { + private CommitAvailabilityMetrics _commitAvailabilityMetrics; + private String _topic; + private String _name; + + public ConsumeService(String name, CompletableFuture topicPartitionResult, ConsumerFactory consumerFactory) + throws ExecutionException, InterruptedException { + _baseConsumer = consumerFactory.baseConsumer(); + _latencySlaMs = consumerFactory.latencySlaMs(); _name = name; - Map consumerPropsOverride = props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) - ? (Map) props.get(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) : new HashMap<>(); - ConsumeServiceConfig config = new ConsumeServiceConfig(props); - String topic = config.getString(ConsumeServiceConfig.TOPIC_CONFIG); - String zkConnect = config.getString(ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG); - String brokerList = config.getString(ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG); - String consumerClassName = config.getString(ConsumeServiceConfig.CONSUMER_CLASS_CONFIG); - _latencySlaMs = config.getInt(ConsumeServiceConfig.LATENCY_SLA_MS_CONFIG); - _latencyPercentileMaxMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_MAX_MS_CONFIG); - _latencyPercentileGranularityMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG); + _adminClient = consumerFactory.adminClient(); _running = new AtomicBoolean(false); - for (String property: NON_OVERRIDABLE_PROPERTIES) { - if (consumerPropsOverride.containsKey(property)) { - throw new ConfigException("Override must not contain " + property + " config."); - } - } - Properties consumerProps = new Properties(); - - // Assign default config. This has the lowest priority. - consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, FALSE); - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); - consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "kmf-consumer"); - consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "kmf-consumer-group-" + new Random().nextInt()); - consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - if (consumerClassName.equals(NewConsumer.class.getCanonicalName()) || consumerClassName.equals(NewConsumer.class.getSimpleName())) { - consumerClassName = NewConsumer.class.getCanonicalName(); - } - - // Assign config specified for ConsumeService. - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); - consumerProps.put(CommonServiceConfig.ZOOKEEPER_CONNECT_CONFIG, zkConnect); - // Assign config specified for consumer. This has the highest priority. - consumerProps.putAll(consumerPropsOverride); - - if (props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG)) { - props.forEach(consumerProps::putIfAbsent); - } - _consumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(topic, consumerProps); topicPartitionResult.thenRun(() -> { MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); List reporters = new ArrayList<>(); reporters.add(new JmxReporter(JMX_PREFIX)); - Metrics metrics = new Metrics(metricConfig, reporters, new SystemTime()); + metrics = new Metrics(metricConfig, reporters, new SystemTime()); Map tags = new HashMap<>(); - tags.put(TAGS_NAME, _name); - _adminClient = AdminClient.create(props); - _sensors = new ConsumeMetrics(metrics, tags, topic, topicPartitionResult); + tags.put(TAGS_NAME, name); + _topic = consumerFactory.topic(); + _sensors = new ConsumeMetrics(metrics, tags, _topic, topicPartitionResult, _adminClient, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); + _commitAvailabilityMetrics = new CommitAvailabilityMetrics(metrics, tags); _consumeThread = new Thread(() -> { try { consume(); } catch (Exception e) { - LOG.error(_name + "/ConsumeService failed", e); + LOG.error(name + "/ConsumeService failed", e); } - }, _name + " consume-service"); + }, name + " consume-service"); _consumeThread.setDaemon(true); - }); + }).get(); } private void consume() throws Exception { - // Delay 1 second to reduce the chance that consumer creates topic before TopicManagementService + /* Delay 1 second to reduce the chance that consumer creates topic before TopicManagementService */ Thread.sleep(1000); Map nextIndexes = new HashMap<>(); @@ -140,27 +90,69 @@ private void consume() throws Exception { while (_running.get()) { BaseConsumerRecord record; try { - record = _consumer.receive(); + record = _baseConsumer.receive(); } catch (Exception e) { _sensors._consumeError.record(); LOG.warn(_name + "/ConsumeService failed to receive record", e); - // Avoid busy while loop + /* Avoid busy while loop */ Thread.sleep(CONSUME_THREAD_SLEEP_MS); continue; } - if (record == null) - continue; + if (record == null) continue; + + GenericRecord avroRecord = null; + try { + avroRecord = Utils.genericRecordFromJson(record.value()); + } catch (Exception exception) { + LOG.error("exception occurred while getting avro record.", exception); + } - GenericRecord avroRecord = Utils.genericRecordFromJson(record.value()); if (avroRecord == null) { _sensors._consumeError.record(); continue; } int partition = record.partition(); + + /* Commit availability and commit latency service */ + try { + /* Call commitAsync, wait for a NON-NULL return value (see https://issues.apache.org/jira/browse/KAFKA-6183) */ + OffsetCommitCallback commitCallback = new OffsetCommitCallback() { + @Override + public void onComplete(Map topicPartitionOffsetAndMetadataMap, Exception kafkaException) { + if (kafkaException != null) { + LOG.error("Exception while trying to perform an asynchronous commit.", kafkaException); + _commitAvailabilityMetrics._failedCommitOffsets.record(); + } else { + _commitAvailabilityMetrics._offsetsCommitted.record(); + } + } + }; + + /* Current timestamp to perform subtraction*/ + long currTimeMillis = System.currentTimeMillis(); + + /* 5 seconds consumer offset commit interval. */ + long timeDiffMillis = TimeUnit.SECONDS.toMillis(COMMIT_TIME_INTERVAL); + + if (currTimeMillis - _baseConsumer.lastCommitted() >= timeDiffMillis) { + /* commit the consumer offset asynchronously with a callback. */ + _baseConsumer.commitAsync(commitCallback); + + /* Record the current time for the committed consumer offset */ + _baseConsumer.updateLastCommit(); + } + + } catch (Exception exception) { + LOG.error("Exception while trying to perform an asynchronous commit.", exception); + _commitAvailabilityMetrics._failedCommitOffsets.record(); + } + /* Finished consumer offset commit service. */ + long index = (Long) avroRecord.get(DefaultTopicSchema.INDEX_FIELD.name()); long currMs = System.currentTimeMillis(); long prevMs = (Long) avroRecord.get(DefaultTopicSchema.TIME_FIELD.name()); + _sensors._recordsConsumed.record(); _sensors._bytesConsumed.record(record.value().length()); _sensors._recordsDelay.record(currMs - prevMs); @@ -174,8 +166,10 @@ record = _consumer.receive(); } long nextIndex = nextIndexes.get(partition); + if (nextIndex == -1 || index == nextIndex) { nextIndexes.put(partition, index + 1); + } else if (index < nextIndex) { _sensors._recordsDuplicated.record(); } else if (index > nextIndex) { @@ -185,6 +179,11 @@ record = _consumer.receive(); LOG.info("_recordsLost recorded: Avro record current index: {} at {}. Next index: {}. Lost {} records.", index, currMs, nextIndex, numLostRecords); } } + /* end of consume() while loop */ + } + + Metrics metrics() { + return metrics; } @Override @@ -199,7 +198,7 @@ public synchronized void start() { public synchronized void stop() { if (_running.compareAndSet(true, false)) { try { - _consumer.close(); + _baseConsumer.close(); } catch (Exception e) { LOG.warn(_name + "/ConsumeService while trying to close consumer.", e); } @@ -217,82 +216,4 @@ public boolean isRunning() { return _running.get() && _consumeThread.isAlive(); } - private class ConsumeMetrics { - private final Sensor _bytesConsumed; - private final Sensor _consumeError; - private final Sensor _recordsConsumed; - private final Sensor _recordsDuplicated; - private final Sensor _recordsLost; - private final Sensor _recordsDelay; - private final Sensor _recordsDelayed; - - ConsumeMetrics(final Metrics metrics, final Map tags, String topicName, CompletableFuture topicPartitionReady) { - topicPartitionReady.thenRun(() -> { - DescribeTopicsResult describeTopicsResult = _adminClient.describeTopics(Collections.singleton(topicName)); - Map> topicResultValues = describeTopicsResult.values(); - KafkaFuture topicDescriptionKafkaFuture = topicResultValues.get(topicName); - TopicDescription topicDescription = null; - try { - topicDescription = topicDescriptionKafkaFuture.get(); - } catch (InterruptedException | ExecutionException e) { - LOG.error("Exception occurred while retrieving the topic description.", e); - } - int partitionCount = topicDescription.partitions().size(); - Sensor topicPartitionCount = metrics.sensor("topic-partitions"); - topicPartitionCount.add( - new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), - new Total(partitionCount)); - }); - - _bytesConsumed = metrics.sensor("bytes-consumed"); - _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); - - _consumeError = metrics.sensor("consume-error"); - _consumeError.add(new MetricName("consume-error-rate", METRIC_GROUP_NAME, "The average number of errors per second", tags), new Rate()); - _consumeError.add(new MetricName("consume-error-total", METRIC_GROUP_NAME, "The total number of errors", tags), new Total()); - - _recordsConsumed = metrics.sensor("records-consumed"); - _recordsConsumed.add(new MetricName("records-consumed-rate", METRIC_GROUP_NAME, "The average number of records per second that are consumed", tags), new Rate()); - _recordsConsumed.add(new MetricName("records-consumed-total", METRIC_GROUP_NAME, "The total number of records that are consumed", tags), new Total()); - - _recordsDuplicated = metrics.sensor("records-duplicated"); - _recordsDuplicated.add(new MetricName("records-duplicated-rate", METRIC_GROUP_NAME, "The average number of records per second that are duplicated", tags), new Rate()); - _recordsDuplicated.add(new MetricName("records-duplicated-total", METRIC_GROUP_NAME, "The total number of records that are duplicated", tags), new Total()); - - _recordsLost = metrics.sensor("records-lost"); - _recordsLost.add(new MetricName("records-lost-rate", METRIC_GROUP_NAME, "The average number of records per second that are lost", tags), new Rate()); - _recordsLost.add(new MetricName("records-lost-total", METRIC_GROUP_NAME, "The total number of records that are lost", tags), new Total()); - - _recordsDelayed = metrics.sensor("records-delayed"); - _recordsDelayed.add(new MetricName("records-delayed-rate", METRIC_GROUP_NAME, "The average number of records per second that are either lost or arrive after maximum allowed latency under SLA", tags), new Rate()); - _recordsDelayed.add(new MetricName("records-delayed-total", METRIC_GROUP_NAME, "The total number of records that are either lost or arrive after maximum allowed latency under SLA", tags), new Total()); - - _recordsDelay = metrics.sensor("records-delay"); - _recordsDelay.add(new MetricName("records-delay-ms-avg", METRIC_GROUP_NAME, "The average latency of records from producer to consumer", tags), new Avg()); - _recordsDelay.add(new MetricName("records-delay-ms-max", METRIC_GROUP_NAME, "The maximum latency of records from producer to consumer", tags), new Max()); - - // There are 2 extra buckets use for values smaller than 0.0 or larger than max, respectively. - int bucketNum = _latencyPercentileMaxMs / _latencyPercentileGranularityMs + 2; - int sizeInBytes = 4 * bucketNum; - _recordsDelay.add(new Percentiles(sizeInBytes, _latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, - new Percentile(new MetricName("records-delay-ms-99th", METRIC_GROUP_NAME, "The 99th percentile latency of records from producer to consumer", tags), 99.0), - new Percentile(new MetricName("records-delay-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile latency of records from producer to consumer", tags), 99.9), - new Percentile(new MetricName("records-delay-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of records from producer to consumer", tags), 99.99))); - - metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), - (config, now) -> { - double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); - - if (new Double(recordsLostRate).isNaN()) - recordsLostRate = 0; - if (new Double(recordsDelayedRate).isNaN()) - recordsDelayedRate = 0; - - return recordsConsumedRate + recordsLostRate > 0 - ? (recordsConsumedRate - recordsDelayedRate) / (recordsConsumedRate + recordsLostRate) : 0; - }); - } - } } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumerFactory.java b/src/main/java/com/linkedin/kmf/services/ConsumerFactory.java new file mode 100644 index 00000000..4a79c421 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ConsumerFactory.java @@ -0,0 +1,32 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + + +package com.linkedin.kmf.services; + +import com.linkedin.kmf.consumer.KMBaseConsumer; +import org.apache.kafka.clients.admin.AdminClient; + + +public interface ConsumerFactory { + + AdminClient adminClient(); + + int latencySlaMs(); + + KMBaseConsumer baseConsumer(); + + String topic(); + + int latencyPercentileMaxMs(); + + int latencyPercentileGranularityMs(); + +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java new file mode 100644 index 00000000..aa65cf77 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java @@ -0,0 +1,113 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import com.linkedin.kmf.consumer.KMBaseConsumer; +import com.linkedin.kmf.consumer.NewConsumer; +import com.linkedin.kmf.services.configs.CommonServiceConfig; +import com.linkedin.kmf.services.configs.ConsumeServiceConfig; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.serialization.StringDeserializer; + + +public class ConsumerFactoryImpl implements ConsumerFactory { + private final KMBaseConsumer _baseConsumer; + private String _topic; + private static final String FALSE = "false"; + private final int _latencyPercentileMaxMs; + private final int _latencyPercentileGranularityMs; + private static final String[] NON_OVERRIDABLE_PROPERTIES = + new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; + private int _latencySlaMs; + private static AdminClient adminClient; + + public ConsumerFactoryImpl(Map props) throws Exception { + adminClient = AdminClient.create(props); + Map consumerPropsOverride = props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) + ? (Map) props.get(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) : new HashMap<>(); + ConsumeServiceConfig config = new ConsumeServiceConfig(props); + _topic = config.getString(ConsumeServiceConfig.TOPIC_CONFIG); + String zkConnect = config.getString(ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG); + String brokerList = config.getString(ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG); + String consumerClassName = config.getString(ConsumeServiceConfig.CONSUMER_CLASS_CONFIG); + _latencySlaMs = config.getInt(ConsumeServiceConfig.LATENCY_SLA_MS_CONFIG); + _latencyPercentileMaxMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_MAX_MS_CONFIG); + _latencyPercentileGranularityMs = config.getInt(ConsumeServiceConfig.LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG); + for (String property: NON_OVERRIDABLE_PROPERTIES) { + if (consumerPropsOverride.containsKey(property)) { + throw new ConfigException("Override must not contain " + property + " config."); + } + } + Properties consumerProps = new Properties(); + + /* Assign default config. This has the lowest priority. */ + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, FALSE); + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "kmf-consumer"); + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "kmf-consumer-group-" + new Random().nextInt()); + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + if (consumerClassName.equals(NewConsumer.class.getCanonicalName()) || consumerClassName.equals(NewConsumer.class.getSimpleName())) { + consumerClassName = NewConsumer.class.getCanonicalName(); + } + + /* Assign config specified for ConsumeService. */ + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); + consumerProps.put(CommonServiceConfig.ZOOKEEPER_CONNECT_CONFIG, zkConnect); + + /* Assign config specified for consumer. This has the highest priority. */ + consumerProps.putAll(consumerPropsOverride); + + if (props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG)) { + props.forEach(consumerProps::putIfAbsent); + } + + _baseConsumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(_topic, consumerProps); + + } + + @Override + public AdminClient adminClient() { + return adminClient; + } + + @Override + public int latencySlaMs() { + return _latencySlaMs; + } + + @Override + public KMBaseConsumer baseConsumer() { + return _baseConsumer; + } + + @Override + public String topic() { + return _topic; + } + + @Override + public int latencyPercentileMaxMs() { + return _latencyPercentileMaxMs; + } + + @Override + public int latencyPercentileGranularityMs() { + return _latencyPercentileGranularityMs; + } + +} diff --git a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java index ac47473f..805ac27d 100644 --- a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterService.java @@ -70,6 +70,8 @@ public void awaitShutdown() { LOG.info("{}/DefaultMetricsReporterService shutdown completed.", _name); } + + private void reportMetrics() { StringBuilder builder = new StringBuilder(); for (String metricName: _metricNames) { diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index a1deb1cb..1e613fee 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -93,6 +93,7 @@ public void awaitShutdown() { LOG.info("{}/KafkaMetricsReporterService shutdown completed", _name); } + private void initializeProducer() { Properties producerProps = new Properties(); producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index db4ca4a7..33781825 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -157,6 +157,8 @@ public void awaitShutdown() { LOG.info("{}/MultiClusterTopicManagementService shutdown completed", _serviceName); } + + private class TopicManagementRunnable implements Runnable { @Override diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index ada601e3..b497d9d5 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -227,6 +227,7 @@ public void awaitShutdown() { LOG.info("{}/ProduceService shutdown completed.", _name); } + @Override public boolean isRunning() { return _running.get() && !_handleNewPartitionsExecutor.isShutdown(); diff --git a/src/main/java/com/linkedin/kmf/services/Service.java b/src/main/java/com/linkedin/kmf/services/Service.java index fb343b58..66e47ea0 100644 --- a/src/main/java/com/linkedin/kmf/services/Service.java +++ b/src/main/java/com/linkedin/kmf/services/Service.java @@ -45,4 +45,9 @@ public interface Service { * Implementations of this method must be thread safe and must be blocking. */ void awaitShutdown(); + + default String getServiceName() { + return this.getClass().getSimpleName(); + } + } diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java index 78b3733d..ee64f22a 100644 --- a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -106,6 +106,7 @@ public void awaitShutdown() { LOG.info("{}/SignalFxMetricsReporterService shutdown completed", _name); } + private SignalFxEndpoint getSignalFxEndpoint(String urlStr) throws Exception { URL url = new URL(urlStr); return new SignalFxEndpoint(url.getProtocol(), url.getHost(), url.getPort()); diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java index f59b7631..703397d9 100644 --- a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterService.java @@ -79,6 +79,7 @@ public void awaitShutdown() { LOG.info("{}/StatsdMetricsReporterService shutdown completed", _name); } + private String generateStatsdMetricName(String bean, String attribute) { String service = bean.split(":")[1]; String serviceName = service.split(",")[0].split("=")[1]; diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index a39e0677..38aad761 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -88,5 +88,6 @@ public boolean isRunning() { public void awaitShutdown() { _multiClusterTopicManagementService.awaitShutdown(); } + } diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index 3c2e0724..c50817d8 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -12,6 +12,7 @@ import com.linkedin.kmf.apps.SingleClusterMonitor; import com.linkedin.kmf.services.ConsumeService; +import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.ProduceService; import com.linkedin.kmf.services.TopicManagementService; import java.util.Map; @@ -47,7 +48,8 @@ public BasicEndToEndTest(Map props, String name) throws Exceptio _topicManagementService = new TopicManagementService(props, name); CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionResult(); _produceService = new ProduceService(props, name); - _consumeService = new ConsumeService(props, name, topicPartitionReady); + ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); + _consumeService = new ConsumeService(name, topicPartitionReady, consumerFactory); } @Override diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index 7a4de10e..e8d16934 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -78,8 +78,8 @@ private KafkaMonitor kafkaMonitor() throws Exception { FakeService.clearCounters(); Map config = new HashMap<>(); Map fakeServiceConfig = new HashMap<>(); - config.put("fake-service", fakeServiceConfig); fakeServiceConfig.put(KafkaMonitor.CLASS_NAME_CONFIG, FakeService.class.getName()); + config.put("fake-service", fakeServiceConfig); return new KafkaMonitor(config); } diff --git a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java new file mode 100644 index 00000000..5d8957b1 --- /dev/null +++ b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java @@ -0,0 +1,187 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import com.linkedin.kmf.common.Utils; +import com.linkedin.kmf.consumer.BaseConsumerRecord; +import com.linkedin.kmf.consumer.KMBaseConsumer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * This public class is a Unit Testing class for the Consume Service Class. + * Also tests for Kafka Monitor Consumer offset commits. + */ +public class ConsumeServiceTest { + private static final String TOPIC = "kafka-monitor-topic-testing"; + private static final Logger LOG = LoggerFactory.getLogger(ConsumeServiceTest.class); + private static final String TAGS_NAME = "name"; + private static final String METRIC_GROUP_NAME = "commit-availability-service"; + /* thread start delay in seconds */ + private static final long THREAD_START_DELAY = 4; + private static final String TAG_NAME_VALUE = "name"; + private static final long MOCK_LAST_COMMITTED_OFFSET = System.currentTimeMillis(); + private static final int PARTITION = 2; + private static final long FIRST_OFFSET = 2; + private static final long SECOND_OFFSET = 3; + + @Test + public void lifecycleTest() throws Exception { + ConsumeService consumeService = consumeService(); + + /* Nothing should be started */ + Assert.assertFalse(consumeService.isRunning()); + Assert.assertNotNull(consumeService.getServiceName()); + + /* Should accept but ignore start because start has not been called */ + consumeService.stop(); + Assert.assertFalse(consumeService.isRunning()); + + /* Should start */ + consumeService.start(); + Assert.assertTrue(consumeService.isRunning()); + + /* Should allow start to be called more than once */ + consumeService.stop(); + consumeService.stop(); + Assert.assertFalse(consumeService.isRunning()); + + /* Should be allowed to shutdown more than once. */ + consumeService.awaitShutdown(); + consumeService.awaitShutdown(); + Assert.assertFalse(consumeService.isRunning()); + } + + @Test + public void commitAvailabilityTest() throws Exception { + ConsumeService consumeService = consumeService(); + + Metrics metrics = consumeService.metrics(); + Map tags = new HashMap<>(); + tags.put(TAGS_NAME, TAG_NAME_VALUE); + + Assert.assertNotNull(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue()); + Assert.assertEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); + + /* Should start */ + consumeService.start(); + Assert.assertTrue(consumeService.isRunning()); + + /* in milliseconds */ + long threadStartDelay = 1000 * THREAD_START_DELAY; + + /* Thread.sleep safe to do here instead of ScheduledExecutorService + * We want to sleep current thread so that consumeService can start running for enough seconds. */ + Thread.sleep(threadStartDelay); + Assert.assertNotNull(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue()); + Assert.assertNotNull(metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).metricValue()); + Assert.assertEquals(metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); + Assert.assertNotEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); + consumeService.stop(); + consumeService.stop(); + + consumeService.awaitShutdown(); + } + + /** + * Sample ConsumeService instance for unit testing + * @return Sample ConsumeService object. + * @throws Exception should the ConsumeService creation fail or throws an error / exception + */ + private ConsumeService consumeService() throws Exception { + LOG.info("Creating an instance of Consume Service for testing.."); + + ConsumerFactory consumerFactory = Mockito.mock(ConsumerFactory.class); + AdminClient adminClient = Mockito.mock(AdminClient.class); + KMBaseConsumer kmBaseConsumer = Mockito.mock(KMBaseConsumer.class); + + Mockito.when(consumerFactory.adminClient()).thenReturn(adminClient); + Mockito.when(consumerFactory.latencySlaMs()).thenReturn(20000); + Mockito.when(consumerFactory.baseConsumer()).thenReturn(kmBaseConsumer); + Mockito.when(consumerFactory.topic()).thenReturn(TOPIC); + + /* LATENCY_PERCENTILE_MAX_MS_CONFIG, */ + Mockito.when(consumerFactory.latencyPercentileMaxMs()).thenReturn(5000); + + /* LATENCY_PERCENTILE_GRANULARITY_MS_CONFIG */ + Mockito.when(consumerFactory.latencyPercentileGranularityMs()).thenReturn(1); + + /* define return value */ + Mockito.when(kmBaseConsumer.lastCommitted()).thenReturn(MOCK_LAST_COMMITTED_OFFSET); + Mockito.when(kmBaseConsumer.committed(Mockito.any())).thenReturn(new OffsetAndMetadata(FIRST_OFFSET)); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) { + OffsetCommitCallback callback = invocationOnMock.getArgument(0); + Map committedOffsets = new HashMap<>(); + committedOffsets.put(new TopicPartition(TOPIC, PARTITION), new OffsetAndMetadata(FIRST_OFFSET)); + callback.onComplete(committedOffsets, null); + + return null; + } + }).when(kmBaseConsumer).commitAsync(Mockito.any(OffsetCommitCallback.class)); + + + /* avro record to KmBaseConsumer record */ + Mockito.when(kmBaseConsumer.receive()).thenReturn( + new BaseConsumerRecord(TOPIC, PARTITION, SECOND_OFFSET, "key", + Utils.jsonFromFields(TOPIC, 2, 6000, "producerId", 2))); + + CompletableFuture topicPartitionResult = new CompletableFuture<>(); + topicPartitionResult.complete(null); + + return new ConsumeService(TAG_NAME_VALUE, topicPartitionResult, consumerFactory); + } + + @Test + public void awaitShutdownOtherThread() throws Exception { + final ConsumeService consumeService = consumeService(); + final AtomicReference error = new AtomicReference<>(); + + Thread thread = new Thread("test awaitshutdown thread") { + @Override + public void run() { + try { + consumeService.awaitShutdown(); + } catch (Throwable t) { + error.set(t); + } + } + }; + + thread.start(); + consumeService.start(); + Thread.sleep(100); + + consumeService.stop(); + thread.join(500); + + Assert.assertFalse(thread.isAlive()); + Assert.assertEquals(error.get(), null); + + } + +} From ede27429c8e301d5aa2bf058b5e2da5e8e0f3075 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 5 Feb 2020 13:28:20 -0800 Subject: [PATCH 080/192] Linkedin's Xinfra Rebranding (#208) * Xinfra rebrand to Xinfra Monitor and Update README.md Signed-off-by: Andrew Choi --- README.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/README.md b/README.md index 00bb6742..95b1ef63 100644 --- a/README.md +++ b/README.md @@ -2,36 +2,36 @@

-# Kafka Monitor +# Xinfra Monitor [![Build Status](https://travis-ci.org/linkedin/kafka-monitor.svg?branch=master)](https://travis-ci.org/linkedin/kafka-monitor) -Kafka Monitor (KMF) is a framework to implement and execute long-running kafka +Xinfra Monitor (formerly Kafka Monitor) is a framework to implement and execute long-running kafka system tests in a real cluster. It complements Kafka’s existing system tests by capturing potential bugs or regressions that are only likely to occur after prolonged period of time or with low probability. Moreover, it allows you to monitor Kafka cluster using end-to-end pipelines to obtain a number of derived vital stats such as end-to-end latency, service availability, consumer offset commit availability, as well as message loss rate. You can easily -deploy Kafka Monitor to test and monitor your Kafka cluster without requiring +deploy Xinfra Monitor to test and monitor your Kafka cluster without requiring any change to your application. -Kafka Monitor can automatically create the monitor topic with the specified config +Xinfra Monitor can automatically create the monitor topic with the specified config and increase partition count of the monitor topic to ensure partition# >= broker#. It can also reassign partition and trigger preferred leader election to ensure that each broker acts as leader of at least one partition of the -monitor topic. This allows Kafka Monitor to detect performance issue on every +monitor topic. This allows Xinfra Monitor to detect performance issue on every broker without requiring users to manually manage the partition assignment of the monitor topic. -Kafka Monitor is used in conjunction with different middle-layer services such as li-apache-kafka-clients in order to monitor single clusters, pipeline desination clusters, and other types of clusters as done in Linkedin engineering for real-time cluster healthchecks. +Xinfra Monitor is used in conjunction with different middle-layer services such as li-apache-kafka-clients in order to monitor single clusters, pipeline desination clusters, and other types of clusters as done in Linkedin engineering for real-time cluster healthchecks. ## Getting Started ### Prerequisites -Kafka Monitor requires Gradle 2.0 or higher. Java 7 should be used for +Xinfra Monitor requires Gradle 2.0 or higher. Java 7 should be used for building in order to support both Java 7 and Java 8 at runtime. -Kafka Monitor supports Apache Kafka 0.8 to 2.0: +Xinfra Monitor supports Apache Kafka 0.8 to 2.0: - Use branch 0.8.2.2 to work with Apache Kafka 0.8 - Use branch 0.9.0.1 to work with Apache Kafka 0.9 - Use branch 0.10.2.1 to work with Apache Kafka 0.10 @@ -44,7 +44,7 @@ Kafka Monitor supports Apache Kafka 0.8 to 2.0: ### Configuration Tips
    -
  1. We advise advanced users to run Kafka Monitor with +
  2. We advise advanced users to run Xinfra Monitor with ./bin/kafka-monitor-start.sh config/kafka-monitor.properties. The default kafka-monitor.properties in the repo provides an simple example of how to monitor a single cluster. You probably need to change the value of @@ -56,17 +56,17 @@ Config class for respective service, e.g. ProduceServiceConfig.java and ConsumeServiceConfig.java.

  3. You can specify multiple SingleClusterMonitor in the kafka-monitor.properties to -monitor multiple Kafka clusters in one Kafka Monitor process. As another +monitor multiple Kafka clusters in one Xinfra Monitor process. As another advanced use-case, you can point ProduceService and ConsumeService to two different Kafka clusters that are connected by MirrorMaker to monitor their end-to-end latency.

  4. -
  5. Kafka Monitor by default will automatically create the monitor topic based on +
  6. Xinfra Monitor by default will automatically create the monitor topic based on the e.g. topic-management.replicationFactor and topic-management.partitionsToBrokersRatio specified in the config. replicationFactor is 1 by default and you probably want to change it to the same replication factor as used for your existing topics. You can disable auto topic creation by setting produce.topic.topicCreationEnabled to false.

  7. -
  8. Kafka Monitor can automatically increase partition count of the monitor topic +
  9. Xinfra Monitor can automatically increase partition count of the monitor topic to ensure partition# >= broker#. It can also reassign partition and trigger preferred leader election to ensure that each broker acts as leader of at least one partition of the monitor topic. To use this feature, use either @@ -83,7 +83,7 @@ EndToEndTest or TopicManagementService in the properties file.
-### Build Kafka Monitor +### Build Xinfra Monitor ``` $ git clone https://github.com/linkedin/kafka-monitor.git $ cd kafka-monitor @@ -95,7 +95,7 @@ $ ./gradlew jar $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties ``` -### Run Kafka Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) +### Run Xinfra Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) Edit `config/kafka-monitor.properties` to specify custom configurations for producer in the key/value map `produce.producer.props` in `config/kafka-monitor.properties`. Similarly specify configurations for consumer as well. The documentation for producer and consumer in the key/value maps can be found in the Apache Kafka wiki. From 5ca6e7e8492902fcf10a7674a2c6236ac7e8aa47 Mon Sep 17 00:00:00 2001 From: Diego Alvarez Date: Tue, 18 Feb 2020 21:03:01 -0500 Subject: [PATCH 081/192] Fix for instantiating ConsumeService (#210) type of the parameter is ConsumerFactory.class rather than ConsumerFactoryImpl.class --- src/main/java/com/linkedin/kmf/KafkaMonitor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index e8d48a8a..b034463f 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -11,6 +11,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.kmf.apps.App; +import com.linkedin.kmf.services.ConsumerFactory; import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.Service; import java.io.BufferedReader; @@ -76,7 +77,7 @@ public KafkaMonitor(Map testProps) throws Exception { completableFuture.complete(null); ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); Service service = (Service) Class.forName(className) - .getConstructor(String.class, CompletableFuture.class, ConsumerFactoryImpl.class) + .getConstructor(String.class, CompletableFuture.class, ConsumerFactory.class) .newInstance(name, completableFuture, consumerFactory); _services.put(name, service); } else { From 37e5469add98cb3dc7419df614fd619a7409f045 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 18 Feb 2020 18:10:48 -0800 Subject: [PATCH 082/192] bump up kafka-clients version. (#209) bump up kafka clients version and usage of cumulative sum Signed-off-by: Andrew Choi --- build.gradle | 2 +- config/log4j2.properties | 2 +- src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 1f672cb8..0af5bf9a 100644 --- a/build.gradle +++ b/build.gradle @@ -42,7 +42,7 @@ allprojects { compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.3.1' + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.0' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' } diff --git a/config/log4j2.properties b/config/log4j2.properties index d44a0b82..96c603b1 100644 --- a/config/log4j2.properties +++ b/config/log4j2.properties @@ -1,4 +1,4 @@ -# Copyright 2016 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this +# Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this # file except in compliance with the License. You may obtain a copy of the License at # # http://www.apache.org/licenses/LICENSE-2.0 diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java index 2a5c83aa..10d84643 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Percentile; import org.apache.kafka.common.metrics.stats.Percentiles; @@ -64,7 +65,7 @@ public class ConsumeMetrics { Sensor topicPartitionCount = metrics.sensor("topic-partitions"); topicPartitionCount.add( new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), - new Total(partitionCount)); + new CumulativeSum(partitionCount)); }); _bytesConsumed = metrics.sensor("bytes-consumed"); From 489f646d0b0ba00eb636c576036ee9105ec9222c Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 19 Feb 2020 15:17:41 -0800 Subject: [PATCH 083/192] correct name for offline-runnable-count (#211) When the application is run, the logs aren't printing out the offline runnable count, and it's due to the wrongly configured report metric name for "kmf:type=kafka-monitor,name=*:offline-runnable-count", Tested with "kmf:type=kafka-monitor:offline-runnable-count", -> and the metric is correctly logged. Signed-off-by: Andrew Choi andchoi@linkedin.com --- config/kafka-monitor.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 97c3ee29..b8a8bcd2 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -82,7 +82,7 @@ "class.name": "com.linkedin.kmf.services.DefaultMetricsReporterService", "report.interval.sec": 1, "report.metrics.list": [ - "kmf:type=kafka-monitor,name=*:offline-runnable-count", + "kmf:type=kafka-monitor:offline-runnable-count", "kmf.services:type=produce-service,name=*:produce-availability-avg", "kmf.services:type=consume-service,name=*:consume-availability-avg", "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", From 204da083c7217abd81ee4d9c7a9f0cacf264bfe0 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 21 Feb 2020 11:14:47 -0800 Subject: [PATCH 084/192] Simplifications of CompletableFuture + Ensuring MultiClusterTopicManagementService performs the topic partitions operations first. (#214) Simplifications of CompletableFuture + Ensuring MultiClusterTopicManagementService performs the topic partitions operations first. Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/KafkaMonitor.java | 2 +- src/main/java/com/linkedin/kmf/apps/App.java | 2 +- .../kmf/apps/MultiClusterMonitor.java | 4 +- .../kmf/apps/SingleClusterMonitor.java | 34 ++++++++++---- .../linkedin/kmf/services/ConsumeMetrics.java | 34 ++------------ .../linkedin/kmf/services/ConsumeService.java | 47 +++++++++++++++++-- .../kmf/services/ConsumerFactoryImpl.java | 4 ++ .../MultiClusterTopicManagementService.java | 9 +--- .../kmf/services/TopicManagementService.java | 8 +--- .../linkedin/kmf/tests/BasicEndToEndTest.java | 4 +- .../kmf/services/ConsumeServiceTest.java | 6 +-- 11 files changed, 86 insertions(+), 68 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index b034463f..4b763e21 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -106,7 +106,7 @@ private boolean constructorContainsFuture(Constructor[] constructors) { return false; } - public synchronized void start() { + public synchronized void start() throws Exception { if (!_isRunning.compareAndSet(false, true)) { return; } diff --git a/src/main/java/com/linkedin/kmf/apps/App.java b/src/main/java/com/linkedin/kmf/apps/App.java index c0b9071d..843ed90a 100644 --- a/src/main/java/com/linkedin/kmf/apps/App.java +++ b/src/main/java/com/linkedin/kmf/apps/App.java @@ -12,7 +12,7 @@ public interface App { - void start(); + void start() throws Exception; void stop(); diff --git a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java index b1fcb0c2..e52e58f7 100644 --- a/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/MultiClusterMonitor.java @@ -76,8 +76,8 @@ private Map createMultiClusterTopicManagementServiceProps(Map topicManagementReady = _multiClusterTopicManagementService.topicManagementResult(); - topicManagementReady.thenRun(() -> { + CompletableFuture topicPartitionResult = _multiClusterTopicManagementService.topicPartitionResult(); + topicPartitionResult.thenRun(() -> { _produceService.start(); _consumeService.start(); }); diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index ec7ba843..338a0d4f 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -11,6 +11,7 @@ package com.linkedin.kmf.apps; import com.linkedin.kmf.services.ConsumeService; +import com.linkedin.kmf.services.ConsumerFactory; import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.DefaultMetricsReporterService; import com.linkedin.kmf.services.JettyService; @@ -29,6 +30,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; @@ -48,6 +51,7 @@ public class SingleClusterMonitor implements App { private static final Logger LOG = LoggerFactory.getLogger(SingleClusterMonitor.class); + private static final int SERVICES_INITIAL_CAPACITY = 4; private final TopicManagementService _topicManagementService; private final ProduceService _produceService; private final ConsumeService _consumeService; @@ -55,27 +59,41 @@ public class SingleClusterMonitor implements App { private final List _allServices; public SingleClusterMonitor(Map props, String name) throws Exception { + ConsumerFactory consumerFactory = new ConsumerFactoryImpl(props); _name = name; _topicManagementService = new TopicManagementService(props, name); - CompletableFuture topicPartitionReady = _topicManagementService.topicPartitionResult(); + CompletableFuture topicPartitionResult = _topicManagementService.topicPartitionResult(); _produceService = new ProduceService(props, name); - ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); - _consumeService = new ConsumeService(name, topicPartitionReady, consumerFactory); - int servicesInitialCapacity = 4; - _allServices = new ArrayList<>(servicesInitialCapacity); + _consumeService = new ConsumeService(name, topicPartitionResult, consumerFactory); + _allServices = new ArrayList<>(SERVICES_INITIAL_CAPACITY); _allServices.add(_topicManagementService); _allServices.add(_produceService); _allServices.add(_consumeService); } @Override - public void start() { + public void start() throws Exception { _topicManagementService.start(); - CompletableFuture completableFuture = _topicManagementService.topicManagementResult(); - completableFuture.thenRun(() -> { + CompletableFuture topicPartitionResult = _topicManagementService.topicPartitionResult(); + try { + /* Delay 2 second to reduce the chance that produce and consumer thread has race condition + with TopicManagementService and MultiClusterTopicManagementService */ + long threadSleepMs = TimeUnit.SECONDS.toMillis(2); + Thread.sleep(threadSleepMs); + } catch (InterruptedException e) { + throw new Exception("Interrupted while sleeping the thread", e); + } + CompletableFuture topicPartitionFuture = topicPartitionResult.thenRun(() -> { _produceService.start(); _consumeService.start(); }); + + try { + topicPartitionFuture.get(); + } catch (InterruptedException | ExecutionException e) { + throw new Exception("Exception occurred while getting the TopicPartitionFuture", e); + } + LOG.info(_name + "/SingleClusterMonitor started."); } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java index 10d84643..d16afb51 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java @@ -10,19 +10,11 @@ package com.linkedin.kmf.services; -import java.util.Collections; import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.DescribeTopicsResult; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Percentile; import org.apache.kafka.common.metrics.stats.Percentiles; @@ -44,29 +36,9 @@ public class ConsumeMetrics { private static final Logger LOG = LoggerFactory.getLogger(ConsumeMetrics.class); ConsumeMetrics(final Metrics metrics, - final Map tags, - String topicName, - CompletableFuture topicPartitionReady, - AdminClient adminClient, - final int latencyPercentileMaxMs, - final int latencyPercentileGranularityMs) { - topicPartitionReady.thenRun(() -> { - DescribeTopicsResult describeTopicsResult = adminClient.describeTopics(Collections.singleton(topicName)); - Map> topicResultValues = describeTopicsResult.values(); - KafkaFuture topicDescriptionKafkaFuture = topicResultValues.get(topicName); - TopicDescription topicDescription = null; - try { - topicDescription = topicDescriptionKafkaFuture.get(); - } catch (InterruptedException | ExecutionException e) { - LOG.error("Exception occurred while retrieving the topic description.", e); - } - - int partitionCount = topicDescription.partitions().size(); - Sensor topicPartitionCount = metrics.sensor("topic-partitions"); - topicPartitionCount.add( - new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), - new CumulativeSum(partitionCount)); - }); + Map tags, + int latencyPercentileMaxMs, + int latencyPercentileGranularityMs) { _bytesConsumed = metrics.sensor("bytes-consumed"); _bytesConsumed.add(new MetricName("bytes-consumed-rate", METRIC_GROUP_NAME, "The average number of bytes per second that are consumed", tags), new Rate()); diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index db20e6cd..edf46fcd 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -15,6 +15,7 @@ import com.linkedin.kmf.consumer.BaseConsumerRecord; import com.linkedin.kmf.consumer.KMBaseConsumer; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -24,16 +25,23 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; public class ConsumeService implements Service { @@ -51,8 +59,12 @@ public class ConsumeService implements Service { private CommitAvailabilityMetrics _commitAvailabilityMetrics; private String _topic; private String _name; + private static final String METRIC_GROUP_NAME = "consume-service"; + private static Map tags; - public ConsumeService(String name, CompletableFuture topicPartitionResult, ConsumerFactory consumerFactory) + public ConsumeService(String name, + CompletableFuture topicPartitionResult, + ConsumerFactory consumerFactory) throws ExecutionException, InterruptedException { _baseConsumer = consumerFactory.baseConsumer(); _latencySlaMs = consumerFactory.latencySlaMs(); @@ -60,15 +72,15 @@ public ConsumeService(String name, CompletableFuture topicPartitionResult, _adminClient = consumerFactory.adminClient(); _running = new AtomicBoolean(false); - topicPartitionResult.thenRun(() -> { + CompletableFuture topicPartitionFuture = topicPartitionResult.thenRun(() -> { MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); List reporters = new ArrayList<>(); reporters.add(new JmxReporter(JMX_PREFIX)); metrics = new Metrics(metricConfig, reporters, new SystemTime()); - Map tags = new HashMap<>(); + tags = new HashMap<>(); tags.put(TAGS_NAME, name); _topic = consumerFactory.topic(); - _sensors = new ConsumeMetrics(metrics, tags, _topic, topicPartitionResult, _adminClient, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); + _sensors = new ConsumeMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); _commitAvailabilityMetrics = new CommitAvailabilityMetrics(metrics, tags); _consumeThread = new Thread(() -> { try { @@ -78,7 +90,9 @@ public ConsumeService(String name, CompletableFuture topicPartitionResult, } }, name + " consume-service"); _consumeThread.setDaemon(true); - }).get(); + }); + + topicPartitionFuture.get(); } private void consume() throws Exception { @@ -186,11 +200,34 @@ Metrics metrics() { return metrics; } + @Test + public synchronized void testStart() { + if (_running.compareAndSet(false, true)) { + _consumeThread.start(); + LOG.info("{}/ConsumeService started.", _name); + } + } + @Override public synchronized void start() { if (_running.compareAndSet(false, true)) { _consumeThread.start(); LOG.info("{}/ConsumeService started.", _name); + + Sensor topicPartitionCount = metrics.sensor("topic-partitions"); + DescribeTopicsResult describeTopicsResult = _adminClient.describeTopics(Collections.singleton(_topic)); + Map> topicResultValues = describeTopicsResult.values(); + KafkaFuture topicDescriptionKafkaFuture = topicResultValues.get(_topic); + TopicDescription topicDescription = null; + try { + topicDescription = topicDescriptionKafkaFuture.get(); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Exception occurred while getting the topicDescriptionKafkaFuture", e); + } + int partitionCount = topicDescription.partitions().size(); + topicPartitionCount.add( + new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), + new CumulativeSum(partitionCount)); } } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java index aa65cf77..49626551 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java @@ -22,6 +22,8 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.StringDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ConsumerFactoryImpl implements ConsumerFactory { @@ -34,8 +36,10 @@ public class ConsumerFactoryImpl implements ConsumerFactory { new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; private int _latencySlaMs; private static AdminClient adminClient; + private static final Logger LOG = LoggerFactory.getLogger(ConsumerFactoryImpl.class); public ConsumerFactoryImpl(Map props) throws Exception { + LOG.info("Creating AdminClient."); adminClient = AdminClient.create(props); Map consumerPropsOverride = props.containsKey(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) ? (Map) props.get(ConsumeServiceConfig.CONSUMER_PROPS_CONFIG) : new HashMap<>(); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 33781825..4e4fb923 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -80,8 +80,6 @@ public class MultiClusterTopicManagementService implements Service { private final long _preferredLeaderElectionIntervalMs; private final ScheduledExecutorService _executor; - private final CompletableFuture _topicManagementResult; - @SuppressWarnings("unchecked") public MultiClusterTopicManagementService(Map props, String serviceName) throws Exception { @@ -93,16 +91,11 @@ public MultiClusterTopicManagementService(Map props, String serv _topicManagementByCluster = initializeTopicManagementHelper(propsByCluster, topic); _scheduleIntervalMs = config.getInt(MultiClusterTopicManagementServiceConfig.REBALANCE_INTERVAL_MS_CONFIG); _preferredLeaderElectionIntervalMs = config.getLong(MultiClusterTopicManagementServiceConfig.PREFERRED_LEADER_ELECTION_CHECK_INTERVAL_MS_CONFIG); - _topicManagementResult = new CompletableFuture<>(); _executor = Executors.newSingleThreadScheduledExecutor( r -> new Thread(r, _serviceName + "-multi-cluster-topic-management-service")); _topicPartitionResult.complete(null); } - public CompletableFuture topicManagementResult() { - return _topicManagementResult; - } - public CompletableFuture topicPartitionResult() { return _topicPartitionResult; } @@ -182,7 +175,7 @@ public void run() { for (TopicManagementHelper helper : _topicManagementByCluster.values()) { helper.maybeAddPartitions(minPartitionNum); } - _topicManagementResult.complete(null); + for (Map.Entry entry : _topicManagementByCluster.entrySet()) { String clusterName = entry.getKey(); TopicManagementHelper helper = entry.getValue(); diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java index 38aad761..302463e9 100644 --- a/src/main/java/com/linkedin/kmf/services/TopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementService.java @@ -23,16 +23,14 @@ */ public class TopicManagementService implements Service { private final MultiClusterTopicManagementService _multiClusterTopicManagementService; - CompletableFuture _topicPartitionResult = new CompletableFuture<>(); public TopicManagementService(Map props, String serviceName) throws Exception { Map serviceProps = createMultiClusterTopicManagementServiceProps(props, serviceName); _multiClusterTopicManagementService = new MultiClusterTopicManagementService(serviceProps, serviceName); - _topicPartitionResult.complete(null); } public CompletableFuture topicPartitionResult() { - return _topicPartitionResult; + return _multiClusterTopicManagementService.topicPartitionResult(); } /** @@ -69,10 +67,6 @@ public synchronized void start() { _multiClusterTopicManagementService.start(); } - public CompletableFuture topicManagementResult() { - return _multiClusterTopicManagementService.topicManagementResult(); - } - @Override public synchronized void stop() { diff --git a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java index c50817d8..b9bb0b32 100644 --- a/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java +++ b/src/main/java/com/linkedin/kmf/tests/BasicEndToEndTest.java @@ -55,8 +55,8 @@ public BasicEndToEndTest(Map props, String name) throws Exceptio @Override public void start() { _topicManagementService.start(); - CompletableFuture completableFuture = _topicManagementService.topicManagementResult(); - completableFuture.thenRun(() -> { + CompletableFuture topicPartitionResult = _topicManagementService.topicPartitionResult(); + topicPartitionResult.thenRun(() -> { try { _produceService.start(); _consumeService.start(); diff --git a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java index 5d8957b1..15081e72 100644 --- a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java @@ -61,7 +61,7 @@ public void lifecycleTest() throws Exception { Assert.assertFalse(consumeService.isRunning()); /* Should start */ - consumeService.start(); + consumeService.testStart(); Assert.assertTrue(consumeService.isRunning()); /* Should allow start to be called more than once */ @@ -87,7 +87,7 @@ public void commitAvailabilityTest() throws Exception { Assert.assertEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); /* Should start */ - consumeService.start(); + consumeService.testStart(); Assert.assertTrue(consumeService.isRunning()); /* in milliseconds */ @@ -173,7 +173,7 @@ public void run() { }; thread.start(); - consumeService.start(); + consumeService.testStart(); Thread.sleep(100); consumeService.stop(); From fcb8b27e40b7f5ab77254c54ca5ee33d81b816f9 Mon Sep 17 00:00:00 2001 From: Sean McCauliff Date: Mon, 2 Mar 2020 10:35:57 -0800 Subject: [PATCH 085/192] Remove some unneeded dependencies espicaly on Jetty. Jetty is only needed for (#220) the JolokiaService, but it will start it's on embedded webserver should someone want to run that service. Narrow the scope of the testng dependency. --- bin/kmf-run-class.sh | 2 +- build.gradle | 5 +- config/kafka-monitor.properties | 5 -- config/multi-cluster-monitor.properties | 5 -- .../kmf/apps/SingleClusterMonitor.java | 10 --- .../linkedin/kmf/services/ConsumeService.java | 5 +- .../linkedin/kmf/services/JettyService.java | 66 ------------------- .../linkedin/kmf/services/JolokiaService.java | 2 +- .../kmf/services/ConsumeServiceTest.java | 6 +- 9 files changed, 7 insertions(+), 99 deletions(-) delete mode 100644 src/main/java/com/linkedin/kmf/services/JettyService.java diff --git a/bin/kmf-run-class.sh b/bin/kmf-run-class.sh index cfaa5109..0fb1f8a2 100755 --- a/bin/kmf-run-class.sh +++ b/bin/kmf-run-class.sh @@ -74,7 +74,7 @@ fi # JVM performance options if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then - KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" + KAFKA_JVM_PERFORMANCE_OPTS="-server -Djava.awt.headless=true" fi diff --git a/build.gradle b/build.gradle index 0af5bf9a..d9c6aebb 100644 --- a/build.gradle +++ b/build.gradle @@ -32,11 +32,8 @@ allprojects { dependencies { compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'org.apache.logging.log4j:log4j-slf4j-impl:2.13.0' - compile 'org.apache.avro:avro:1.4.0' - compile 'org.testng:testng:6.8.8' - compile 'org.eclipse.jetty:jetty-server:8.1.19.v20160209' + compile 'org.apache.avro:avro:1.4.1' compile 'org.json:json:20140107' - compile 'com.fasterxml.jackson.core:jackson-databind:2.7.1' compile 'org.jolokia:jolokia-jvm:1.6.2' compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index b8a8bcd2..4b629ff4 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -69,11 +69,6 @@ } }, - "jetty-service": { - "class.name": "com.linkedin.kmf.services.JettyService", - "jetty.port": 8000 - }, - "jolokia-service": { "class.name": "com.linkedin.kmf.services.JolokiaService" }, diff --git a/config/multi-cluster-monitor.properties b/config/multi-cluster-monitor.properties index 41c5f975..dd40b035 100644 --- a/config/multi-cluster-monitor.properties +++ b/config/multi-cluster-monitor.properties @@ -86,11 +86,6 @@ ] }, - "jetty-service": { - "class.name": "com.linkedin.kmf.services.JettyService", - "jetty.port": 8000 - }, - "jolokia-service": { "class.name": "com.linkedin.kmf.services.JolokiaService" } diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 338a0d4f..0ed1cf56 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -14,7 +14,6 @@ import com.linkedin.kmf.services.ConsumerFactory; import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.DefaultMetricsReporterService; -import com.linkedin.kmf.services.JettyService; import com.linkedin.kmf.services.JolokiaService; import com.linkedin.kmf.services.ProduceService; import com.linkedin.kmf.services.Service; @@ -343,14 +342,5 @@ public static void main(String[] args) throws Exception { JolokiaService jolokiaService = new JolokiaService(new HashMap<>(), "end-to-end"); jolokiaService.start(); - - JettyService jettyService = new JettyService(new HashMap<>(), "end-to-end"); - jettyService.start(); - - if (!app.isRunning()) { - LOG.error("Some services have stopped."); - System.exit(-1); - } - app.awaitShutdown(); } } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index edf46fcd..25fbd0ea 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -41,8 +41,6 @@ import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.annotations.Test; - public class ConsumeService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ConsumeService.class); @@ -200,8 +198,7 @@ Metrics metrics() { return metrics; } - @Test - public synchronized void testStart() { + void startConsumeThreadForTesting() { if (_running.compareAndSet(false, true)) { _consumeThread.start(); LOG.info("{}/ConsumeService started.", _name); diff --git a/src/main/java/com/linkedin/kmf/services/JettyService.java b/src/main/java/com/linkedin/kmf/services/JettyService.java deleted file mode 100644 index 9cd30c75..00000000 --- a/src/main/java/com/linkedin/kmf/services/JettyService.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this - * file except in compliance with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - */ - -package com.linkedin.kmf.services; - -import com.linkedin.kmf.services.configs.JettyServiceConfig; -import java.util.Map; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.ResourceHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -// Jetty server that serves html files. -public class JettyService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(JettyService.class); - - private final String _name; - private final Server _jettyServer; - private final int _port; - - public JettyService(Map props, String name) { - _name = name; - JettyServiceConfig config = new JettyServiceConfig(props); - _port = config.getInt(JettyServiceConfig.PORT_CONFIG); - _jettyServer = new Server(_port); - ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setDirectoriesListed(true); - resourceHandler.setWelcomeFiles(new String[]{"index.html"}); - resourceHandler.setResourceBase("webapp"); - _jettyServer.setHandler(resourceHandler); - } - - public synchronized void start() { - try { - _jettyServer.start(); - LOG.info("{}/JettyService started at port {}", _name, _port); - } catch (Exception e) { - LOG.error(_name + "/JettyService failed to start", e); - } - } - - public synchronized void stop() { - try { - _jettyServer.stop(); - LOG.info("{}/JettyService stopped", _name); - } catch (Exception e) { - LOG.error(_name + "/JettyService failed to stop", e); - } - } - - public boolean isRunning() { - return _jettyServer.isRunning(); - } - - public void awaitShutdown() { - - } - -} diff --git a/src/main/java/com/linkedin/kmf/services/JolokiaService.java b/src/main/java/com/linkedin/kmf/services/JolokiaService.java index 053e9108..d95cb8d5 100644 --- a/src/main/java/com/linkedin/kmf/services/JolokiaService.java +++ b/src/main/java/com/linkedin/kmf/services/JolokiaService.java @@ -21,7 +21,7 @@ * Jolokia server allows user to query jmx metric value with HTTP request */ public class JolokiaService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(JettyService.class); + private static final Logger LOG = LoggerFactory.getLogger(JolokiaService.class); private final String _name; private final JolokiaServer _jolokiaServer; diff --git a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java index 15081e72..85368483 100644 --- a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java @@ -61,7 +61,7 @@ public void lifecycleTest() throws Exception { Assert.assertFalse(consumeService.isRunning()); /* Should start */ - consumeService.testStart(); + consumeService.startConsumeThreadForTesting(); Assert.assertTrue(consumeService.isRunning()); /* Should allow start to be called more than once */ @@ -87,7 +87,7 @@ public void commitAvailabilityTest() throws Exception { Assert.assertEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); /* Should start */ - consumeService.testStart(); + consumeService.startConsumeThreadForTesting(); Assert.assertTrue(consumeService.isRunning()); /* in milliseconds */ @@ -173,7 +173,7 @@ public void run() { }; thread.start(); - consumeService.testStart(); + consumeService.startConsumeThreadForTesting(); Thread.sleep(100); consumeService.stop(); From e8a42080a66f4ecf2d764749399ef1d63e5954be Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 2 Mar 2020 12:23:54 -0800 Subject: [PATCH 086/192] expose export offset committed avg (#221) expose export offset committed avg for importing into internal xinfra monitor Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 0ed1cf56..4be6e411 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -323,6 +323,7 @@ public static void main(String[] args) throws Exception { "kmf.services:type=consume-service,name=*:topic-partitions-count", "kmf.services:type=produce-service,name=*:produce-availability-avg", "kmf.services:type=consume-service,name=*:consume-availability-avg", + "kmf.services:type=consume-service,name=*:offsets-committed-avg", "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", "kmf.services:type=consume-service,name=*:records-lost-total", From 9f1e4bf40b0c005545fe667a09c1aea4025eb831 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 2 Mar 2020 17:11:36 -0800 Subject: [PATCH 087/192] commit availability sensor right types (#222) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Xinfra Monitor change commit-availability sensor to correct type Signed-off-by: Andrew Choi andchoi@linkedin.com 🙂 --- .../java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 4be6e411..f8017bf4 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -323,7 +323,6 @@ public static void main(String[] args) throws Exception { "kmf.services:type=consume-service,name=*:topic-partitions-count", "kmf.services:type=produce-service,name=*:produce-availability-avg", "kmf.services:type=consume-service,name=*:consume-availability-avg", - "kmf.services:type=consume-service,name=*:offsets-committed-avg", "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", "kmf.services:type=consume-service,name=*:records-lost-total", @@ -333,8 +332,8 @@ public static void main(String[] args) throws Exception { "kmf.services:type=produce-service,name=*:records-produced-rate", "kmf.services:type=produce-service,name=*:produce-error-rate", "kmf.services:type=consume-service,name=*:consume-error-rate", - "kmf.services:type=consume-service,name=*:commit-latency-avg", - "kmf.services:type=consume-service,name=*:commit-availability-avg" + "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", + "kmf.services:type=commit-availability-service,name=*:commit-latency-avg" ); props.put(DefaultMetricsReporterServiceConfig.REPORT_METRICS_CONFIG, metrics); From f1fdc42303d4e4e7a6f08939517d2666c39477f6 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 4 Mar 2020 10:33:08 -0800 Subject: [PATCH 088/192] [ kafka-monitor ] close ZK connections cleanly (#223) Removes a regression due to which ZK connections opened by MultiClusterTopicManagementService aren't recycled. This commit attempts to fix that issue by wrapping auto-closable KafkaZkClient with try-with-resources statement. --- .../MultiClusterTopicManagementService.java | 129 ++++++++++-------- 1 file changed, 70 insertions(+), 59 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 4e4fb923..2df1ec43 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -322,62 +322,71 @@ private Set getAvailableBrokers() throws ExecutionException, InterruptedEx } void maybeReassignPartitionAndElectLeader() throws Exception { - KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); - - List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); - Collection brokers = this.getAvailableBrokers(); - boolean partitionReassigned = false; - if (partitionInfoList.size() == 0) { - throw new IllegalStateException("Topic " + _topic + " does not exist in cluster."); - } + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { + + List partitionInfoList = _adminClient + .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + Collection brokers = this.getAvailableBrokers(); + boolean partitionReassigned = false; + if (partitionInfoList.size() == 0) { + throw new IllegalStateException("Topic " + _topic + " does not exist in cluster."); + } - int currentReplicationFactor = getReplicationFactor(partitionInfoList); - int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); + int currentReplicationFactor = getReplicationFactor(partitionInfoList); + int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); - if (_replicationFactor < currentReplicationFactor) - LOG.debug("Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", - _replicationFactor, currentReplicationFactor, _topic); + if (_replicationFactor < currentReplicationFactor) + LOG.debug( + "Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", + _replicationFactor, currentReplicationFactor, _topic); - if (expectedReplicationFactor > currentReplicationFactor && !zkClient.reassignPartitionsInProgress()) { - LOG.info("MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" - + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); - reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); - partitionReassigned = true; - } + if (expectedReplicationFactor > currentReplicationFactor && !zkClient + .reassignPartitionsInProgress()) { + LOG.info( + "MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" + + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), + expectedReplicationFactor); + partitionReassigned = true; + } - // Update the properties of the monitor topic if any config is different from the user-specified config - Properties currentProperties = zkClient.getEntityConfigs(ConfigType.Topic(), _topic); - Properties expectedProperties = new Properties(); - for (Object key: currentProperties.keySet()) - expectedProperties.put(key, currentProperties.get(key)); - for (Object key: _topicProperties.keySet()) - expectedProperties.put(key, _topicProperties.get(key)); - - if (!currentProperties.equals(expectedProperties)) { - LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " - + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); - zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); - } + // Update the properties of the monitor topic if any config is different from the user-specified config + Properties currentProperties = zkClient.getEntityConfigs(ConfigType.Topic(), _topic); + Properties expectedProperties = new Properties(); + for (Object key : currentProperties.keySet()) + expectedProperties.put(key, currentProperties.get(key)); + for (Object key : _topicProperties.keySet()) + expectedProperties.put(key, _topicProperties.get(key)); + + if (!currentProperties.equals(expectedProperties)) { + LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); + zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); + } - if (partitionInfoList.size() >= brokers.size() && - someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient.reassignPartitionsInProgress()) { - LOG.info("{} will reassign partitions of the topic {} in cluster.", this.getClass().toString(), _topic); - reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); - partitionReassigned = true; - } + if (partitionInfoList.size() >= brokers.size() && + someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient + .reassignPartitionsInProgress()) { + LOG.info("{} will reassign partitions of the topic {} in cluster.", + this.getClass().toString(), _topic); + reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), + expectedReplicationFactor); + partitionReassigned = true; + } - if (partitionInfoList.size() >= brokers.size() && - someBrokerNotElectedLeader(partitionInfoList, brokers)) { - if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { - LOG.info( - "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " - + "cluster.", _topic - ); - triggerPreferredLeaderElection(partitionInfoList, _topic); - _preferredLeaderElectionRequested = false; - } else { - _preferredLeaderElectionRequested = true; + if (partitionInfoList.size() >= brokers.size() && + someBrokerNotElectedLeader(partitionInfoList, brokers)) { + if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { + LOG.info( + "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " + + "cluster.", _topic + ); + triggerPreferredLeaderElection(partitionInfoList, _topic); + _preferredLeaderElectionRequested = false; + } else { + _preferredLeaderElectionRequested = true; + } } } } @@ -387,15 +396,17 @@ void maybeElectLeader() throws Exception { return; } - KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null); - - if (!zkClient.reassignPartitionsInProgress()) { - List partitionInfoList = _adminClient.describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); - LOG.info("MultiClusterTopicManagementService will trigger requested preferred leader election for the" - + " topic {} in cluster.", _topic); - triggerPreferredLeaderElection(partitionInfoList, _topic); - _preferredLeaderElectionRequested = false; + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { + if (!zkClient.reassignPartitionsInProgress()) { + List partitionInfoList = _adminClient + .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); + LOG.info( + "MultiClusterTopicManagementService will trigger requested preferred leader election for the" + + " topic {} in cluster.", _topic); + triggerPreferredLeaderElection(partitionInfoList, _topic); + _preferredLeaderElectionRequested = false; + } } } From de8e1f91eb021547722d4692b973722c00a6f08a Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 4 Mar 2020 12:57:57 -0800 Subject: [PATCH 089/192] Fix log4j2 conf to use kafka-client.log file (#217) - Remove misconfigured 'stdout' appender - Fix 'kafkaClientAppender' appender - Fix use of system properties - Add configuration to use 'kafkaClientAppender' for kafka logs --- config/log4j2.properties | 47 ++++++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/config/log4j2.properties b/config/log4j2.properties index 96c603b1..b651e27d 100644 --- a/config/log4j2.properties +++ b/config/log4j2.properties @@ -7,35 +7,40 @@ # an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. status = error +dest = err name = PropertiesConfig -filters = threshold filter.threshold.type = ThresholdFilter filter.threshold.level = debug -additivity.com.linkedin.kmf.core.KafkaMonitor = false -additivity.org.apache.kafka = false -additivity.kafka = false - -appenders = console - appender.console.type = Console appender.console.name = STDOUT appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n +appender.console.layout.pattern = [%d] %p %m (%c)%n -appender.stdout=org.apache.log4j.ConsoleAppender -appender.stdout.layout=org.apache.log4j.PatternLayout -appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n +appender.kafka.type = RollingFile +appender.kafka.name = KAFKA +appender.kafka.filename = ${sys:kafka.logs.dir}/kafka-client.log +appender.kafka.filePattern = ${sys:kafka.logs.dir}/kafka-client.log.%d{yyyy-MM-dd-HH} +appender.kafka.layout.type = PatternLayout +appender.kafka.layout.pattern = [%d] %p %m (%c)%n +appender.kafka.policies.type = Policies +appender.kafka.policies.time.type = TimeBasedTriggeringPolicy -appender.kafkaClientAppender=org.apache.log4j.DailyRollingFileAppender -appender.kafkaClientAppender.DatePattern='.'yyyy-MM-dd-HH -appender.kafkaClientAppender.File=${kafka.logs.dir}/kafka-client.log -appender.kafkaClientAppender.layout=org.apache.log4j.PatternLayout -appender.kafkaClientAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -rootLogger = INFO rootLogger.level = info -rootLogger.appenderRefs = stdout -rootLogger.appenderRef.stdout.ref = STDOUT - +rootLogger.appenderRef.console.ref = STDOUT + +logger.kmf.name = com.linkedin.kmf.core.KafkaMonitor +logger.kmf.level = info +logger.kmf.additivity = false +logger.kmf.appenderRef.console.ref = STDOUT + +logger.kafkaClient.name = org.apache.kafka +logger.kafkaClient.level = warn +logger.kafkaClient.additivity = false +logger.kafkaClient.appenderRef.kafka.ref = KAFKA + +logger.kafka.name = kafka +logger.kafka.level = warn +logger.kafka.additivity = false +logger.kafka.appenderRef.kafka.ref = KAFKA From 4134d70fec156ea395fdfb4fa2fdf2092931b972 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 4 Mar 2020 13:16:02 -0800 Subject: [PATCH 090/192] "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", (#224) Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index f8017bf4..453f82b4 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -332,7 +332,10 @@ public static void main(String[] args) throws Exception { "kmf.services:type=produce-service,name=*:records-produced-rate", "kmf.services:type=produce-service,name=*:produce-error-rate", "kmf.services:type=consume-service,name=*:consume-error-rate", + "kmf.services:type=commit-availability-service,name=*:offsets-committed-total", "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-total", + "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", "kmf.services:type=commit-availability-service,name=*:commit-latency-avg" ); props.put(DefaultMetricsReporterServiceConfig.REPORT_METRICS_CONFIG, metrics); From c23ad615675aee370dac20f59f49790beb383f0a Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 5 Mar 2020 09:35:02 -0800 Subject: [PATCH 091/192] CommitLatencyMetrics and Corresponding Initial Unit Testing (#207) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CommitLatencyMetrics and Corresponding Initial Unit Testing Tested by locally deploying with zookeeper and kafka server instance. (Note, the implementation for latency metrics is in a separate PR) Signed-off-by: Andrew Choi 🌿 --- .../services/CommitAvailabilityMetrics.java | 12 ++- .../kmf/services/CommitLatencyMetrics.java | 54 ++++++++++++ .../linkedin/kmf/services/ConsumeService.java | 2 +- .../kmf/services/ConsumeServiceTest.java | 88 +++++++++++++++---- 4 files changed, 135 insertions(+), 21 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java diff --git a/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java b/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java index 30a2a2d4..2af386dc 100644 --- a/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/CommitAvailabilityMetrics.java @@ -47,9 +47,15 @@ class CommitAvailabilityMetrics { metrics.addMetric(new MetricName("offsets-committed-avg", METRIC_GROUP_NAME, "The average offset commits availability.", tags), (MetricConfig config, long now) -> { - double offsetsCommittedCount = metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).value(); - double offsetsCommittedErrorCount = metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).value(); - return offsetsCommittedCount / (offsetsCommittedCount + offsetsCommittedErrorCount); + Object offsetCommitTotal = metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(); + Object offsetCommitFailTotal = metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).metricValue(); + if (offsetCommitTotal != null && offsetCommitFailTotal != null) { + double offsetsCommittedCount = (double) offsetCommitTotal; + double offsetsCommittedErrorCount = (double) offsetCommitFailTotal; + return offsetsCommittedCount / (offsetsCommittedCount + offsetsCommittedErrorCount); + } else { + return 0; + } }); } } diff --git a/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java new file mode 100644 index 00000000..8b838976 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java @@ -0,0 +1,54 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Percentile; +import org.apache.kafka.common.metrics.stats.Percentiles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * The CommitLatencyMetrics class contains methods that measures and + * determines the latency of Kafka consumer offset commit(). + */ +public class CommitLatencyMetrics { + private static final String METRIC_GROUP_NAME = "commit-latency-service"; + private static final Logger LOG = LoggerFactory.getLogger(CommitLatencyMetrics.class); + private final Sensor _commitOffsetLatency; + + /** + * Metrics for Calculating the offset commit latency of a consumer. + * @param metrics the commit offset metrics + * @param tags the tags associated, i.e) kmf.services:name=single-cluster-monitor + */ + CommitLatencyMetrics(Metrics metrics, Map tags, int latencyPercentileMaxMs, int latencyPercentileGranularityMs) { + _commitOffsetLatency = metrics.sensor("commit-offset-latency"); + _commitOffsetLatency.add(new MetricName("commit-offset-latency-ms-avg", METRIC_GROUP_NAME, "The average latency in ms of committing offset", tags), new Avg()); + _commitOffsetLatency.add(new MetricName("commit-offset-latency-ms-max", METRIC_GROUP_NAME, "The maximum latency in ms of committing offset", tags), new Max()); + + int bucketNum = latencyPercentileMaxMs / latencyPercentileGranularityMs + 2; + int sizeInBytes = bucketNum * 4; + _commitOffsetLatency.add(new Percentiles(sizeInBytes, latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, + new Percentile(new MetricName("commit-offset-latency-ms-99th", METRIC_GROUP_NAME, "The 99th percentile latency of committing offset", tags), 99.0), + new Percentile(new MetricName("commit-offset-latency-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile latency of committing offset", tags), 99.9), + new Percentile(new MetricName("commit-offset-latency-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of committing offset", tags), 99.99))); + + LOG.info("{} was constructed successfully.", this.getClass().getSimpleName()); + + } +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 25fbd0ea..22b8ed17 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -221,7 +221,7 @@ public synchronized void start() { } catch (InterruptedException | ExecutionException e) { LOG.error("Exception occurred while getting the topicDescriptionKafkaFuture", e); } - int partitionCount = topicDescription.partitions().size(); + double partitionCount = topicDescription.partitions().size(); topicPartitionCount.add( new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), new CumulativeSum(partitionCount)); diff --git a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java index 85368483..57dc8028 100644 --- a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java @@ -16,6 +16,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -28,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -47,6 +49,7 @@ public class ConsumeServiceTest { private static final int PARTITION = 2; private static final long FIRST_OFFSET = 2; private static final long SECOND_OFFSET = 3; + private static Map tags; @Test public void lifecycleTest() throws Exception { @@ -64,24 +67,13 @@ public void lifecycleTest() throws Exception { consumeService.startConsumeThreadForTesting(); Assert.assertTrue(consumeService.isRunning()); - /* Should allow start to be called more than once */ - consumeService.stop(); - consumeService.stop(); - Assert.assertFalse(consumeService.isRunning()); - - /* Should be allowed to shutdown more than once. */ - consumeService.awaitShutdown(); - consumeService.awaitShutdown(); - Assert.assertFalse(consumeService.isRunning()); + shutdownConsumeService(consumeService); } @Test public void commitAvailabilityTest() throws Exception { ConsumeService consumeService = consumeService(); - - Metrics metrics = consumeService.metrics(); - Map tags = new HashMap<>(); - tags.put(TAGS_NAME, TAG_NAME_VALUE); + Metrics metrics = consumeServiceMetrics(consumeService); Assert.assertNotNull(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue()); Assert.assertEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); @@ -97,13 +89,36 @@ public void commitAvailabilityTest() throws Exception { * We want to sleep current thread so that consumeService can start running for enough seconds. */ Thread.sleep(threadStartDelay); Assert.assertNotNull(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue()); - Assert.assertNotNull(metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).metricValue()); + Assert.assertNotNull(metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, + tags)).metricValue()); Assert.assertEquals(metrics.metrics().get(metrics.metricName("failed-commit-offsets-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); Assert.assertNotEquals(metrics.metrics().get(metrics.metricName("offsets-committed-total", METRIC_GROUP_NAME, tags)).metricValue(), 0.0); - consumeService.stop(); - consumeService.stop(); + shutdownConsumeService(consumeService); + } - consumeService.awaitShutdown(); + @Test + public void commitLatencyTest() throws Exception { + CommitLatencyMetrics commitLatencyMetrics = Mockito.mock(CommitLatencyMetrics.class); + Assert.assertNotNull(commitLatencyMetrics); + + ConsumeService consumeService = consumeService(); + Metrics metrics = consumeServiceMetrics(consumeService); + + Assert.assertNull(metrics.metrics().get(metrics.metricName("commit-offset-latency-ms-avg", METRIC_GROUP_NAME, tags))); + Assert.assertNull(metrics.metrics().get(metrics.metricName("commit-offset-latency-ms-max", METRIC_GROUP_NAME, tags))); + + /* Should start */ + consumeService.startConsumeThreadForTesting(); + Assert.assertTrue(consumeService.isRunning()); + + /* in milliseconds */ + long threadStartDelay = TimeUnit.SECONDS.toMillis(THREAD_START_DELAY); + + /* Thread.sleep safe to do here instead of ScheduledExecutorService + * We want to sleep current thread so that consumeService can start running for enough seconds. */ + Thread.sleep(threadStartDelay); + + shutdownConsumeService(consumeService); } /** @@ -184,4 +199,43 @@ public void run() { } + /** + * return consume service metrics. + * @param consumeService ConsumeService object + * @return consume service metrics + */ + private Metrics consumeServiceMetrics(ConsumeService consumeService) { + setup(); + Metrics metrics = consumeService.metrics(); + return metrics; + } + + /** + * set up the tags for the metrics + */ + @BeforeMethod + public void setup() { + tags = new HashMap<>(); + tags.put(TAGS_NAME, TAG_NAME_VALUE); + } + + /** + * shutdown the consume service. + * @param consumeService object of ConsumeService + */ + private void shutdownConsumeService(ConsumeService consumeService) { + /* + intentionally attempt stopping twice as such executions shouldn't throw any exceptions. + Should allow start to be called more than once + */ + consumeService.stop(); + consumeService.stop(); + Assert.assertFalse(consumeService.isRunning()); + + /* Should be allowed to shutdown more than once. */ + consumeService.awaitShutdown(); + consumeService.awaitShutdown(); + Assert.assertFalse(consumeService.isRunning()); + } + } From 14aa2858117c4855561f609ddb2f3937e5a9296f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 6 Mar 2020 12:43:04 -0800 Subject: [PATCH 092/192] Measurement of Commit Latency Service Metrics (#225) Measurement of Commit Latency Service Metrics 1 - Adds setCommitStartTimeMs() functionality 2 - Adds public void recordCommitComplete() for finishing the recording of consumer offset commit. 3 - Implements public void setCommitStartTimeMs(long time) for setting in milliseconds the start time of consumer offset commit. 4 - Implements public long commitStartTimeMs() for retrieving the start time of consumer offset commit. --- .../kmf/services/CommitLatencyMetrics.java | 53 ++++++++++++++++++- .../linkedin/kmf/services/ConsumeService.java | 13 ++--- .../GraphiteMetricsReporterService.java | 16 +++--- .../kmf/services/ConsumeServiceTest.java | 6 +-- 4 files changed, 72 insertions(+), 16 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java index 8b838976..33c22385 100644 --- a/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java @@ -30,6 +30,8 @@ public class CommitLatencyMetrics { private static final String METRIC_GROUP_NAME = "commit-latency-service"; private static final Logger LOG = LoggerFactory.getLogger(CommitLatencyMetrics.class); private final Sensor _commitOffsetLatency; + private long _commitStartTimeMs; + private volatile boolean _inProgressCommit; /** * Metrics for Calculating the offset commit latency of a consumer. @@ -37,18 +39,67 @@ public class CommitLatencyMetrics { * @param tags the tags associated, i.e) kmf.services:name=single-cluster-monitor */ CommitLatencyMetrics(Metrics metrics, Map tags, int latencyPercentileMaxMs, int latencyPercentileGranularityMs) { + _inProgressCommit = false; _commitOffsetLatency = metrics.sensor("commit-offset-latency"); _commitOffsetLatency.add(new MetricName("commit-offset-latency-ms-avg", METRIC_GROUP_NAME, "The average latency in ms of committing offset", tags), new Avg()); _commitOffsetLatency.add(new MetricName("commit-offset-latency-ms-max", METRIC_GROUP_NAME, "The maximum latency in ms of committing offset", tags), new Max()); + if (latencyPercentileGranularityMs == 0) { + throw new IllegalArgumentException("The latency percentile granularity was incorrectly passed a zero value."); + } + + // 2 extra buckets exist which are respectively designated for values which are less than 0.0 or larger than max. int bucketNum = latencyPercentileMaxMs / latencyPercentileGranularityMs + 2; int sizeInBytes = bucketNum * 4; _commitOffsetLatency.add(new Percentiles(sizeInBytes, latencyPercentileMaxMs, Percentiles.BucketSizing.CONSTANT, new Percentile(new MetricName("commit-offset-latency-ms-99th", METRIC_GROUP_NAME, "The 99th percentile latency of committing offset", tags), 99.0), new Percentile(new MetricName("commit-offset-latency-ms-999th", METRIC_GROUP_NAME, "The 99.9th percentile latency of committing offset", tags), 99.9), new Percentile(new MetricName("commit-offset-latency-ms-9999th", METRIC_GROUP_NAME, "The 99.99th percentile latency of committing offset", tags), 99.99))); - LOG.info("{} was constructed successfully.", this.getClass().getSimpleName()); + } + /** + * start the recording of consumer offset commit + * @throws Exception if the offset commit is already in progress. + */ + public void recordCommitStart() throws Exception { + if (!_inProgressCommit) { + this.setCommitStartTimeMs(System.currentTimeMillis()); + _inProgressCommit = true; + } else { + // inProgressCommit is already set to TRUE; + throw new Exception("Offset commit is already in progress."); + } + } + + /** + * finish the recording of consumer offset commit + */ + public void recordCommitComplete() { + if (_inProgressCommit) { + long commitCompletedMs = System.currentTimeMillis(); + long commitStartMs = this.commitStartTimeMs(); + this._commitOffsetLatency.record(commitCompletedMs - commitStartMs); + _inProgressCommit = false; + } else { + // inProgressCommit is already set to FALSE; + LOG.error("Offset commit is not in progress. CommitLatencyMetrics shouldn't completing a record commit here."); + } + } + + /** + * set in milliseconds the start time of consumer offset commit + * @param time commit start time in ms + */ + public void setCommitStartTimeMs(long time) { + _commitStartTimeMs = time; + } + + /** + * retrieve the start time of consumer offset commit + * @return _commitStartTimeMs + */ + public long commitStartTimeMs() { + return _commitStartTimeMs; } } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 22b8ed17..25ce2aad 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -50,11 +50,12 @@ public class ConsumeService implements Service { private static Metrics metrics; private final AtomicBoolean _running; private final KMBaseConsumer _baseConsumer; - private int _latencySlaMs; + private final int _latencySlaMs; private ConsumeMetrics _sensors; private Thread _consumeThread; - private AdminClient _adminClient; + private final AdminClient _adminClient; private CommitAvailabilityMetrics _commitAvailabilityMetrics; + private CommitLatencyMetrics _commitLatencyMetrics; private String _topic; private String _name; private static final String METRIC_GROUP_NAME = "consume-service"; @@ -79,6 +80,7 @@ public ConsumeService(String name, tags.put(TAGS_NAME, name); _topic = consumerFactory.topic(); _sensors = new ConsumeMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); + _commitLatencyMetrics = new CommitLatencyMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); _commitAvailabilityMetrics = new CommitAvailabilityMetrics(metrics, tags); _consumeThread = new Thread(() -> { try { @@ -125,7 +127,6 @@ record = _baseConsumer.receive(); continue; } int partition = record.partition(); - /* Commit availability and commit latency service */ try { /* Call commitAsync, wait for a NON-NULL return value (see https://issues.apache.org/jira/browse/KAFKA-6183) */ @@ -137,6 +138,7 @@ public void onComplete(Map topicPartitionOffs _commitAvailabilityMetrics._failedCommitOffsets.record(); } else { _commitAvailabilityMetrics._offsetsCommitted.record(); + _commitLatencyMetrics.recordCommitComplete(); } } }; @@ -144,17 +146,16 @@ public void onComplete(Map topicPartitionOffs /* Current timestamp to perform subtraction*/ long currTimeMillis = System.currentTimeMillis(); - /* 5 seconds consumer offset commit interval. */ + /* 4 seconds consumer offset commit interval. */ long timeDiffMillis = TimeUnit.SECONDS.toMillis(COMMIT_TIME_INTERVAL); if (currTimeMillis - _baseConsumer.lastCommitted() >= timeDiffMillis) { /* commit the consumer offset asynchronously with a callback. */ _baseConsumer.commitAsync(commitCallback); - + _commitLatencyMetrics.recordCommitStart(); /* Record the current time for the committed consumer offset */ _baseConsumer.updateLastCommit(); } - } catch (Exception exception) { LOG.error("Exception while trying to perform an asynchronous commit.", exception); _commitAvailabilityMetrics._failedCommitOffsets.record(); diff --git a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java index 00c3d295..f20536ee 100644 --- a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterService.java @@ -51,13 +51,17 @@ public GraphiteMetricsReporterService(Map props, String name) @Override public synchronized void start() { - _executor.scheduleAtFixedRate(() -> { - try { - reportMetrics(); - } catch (Exception e) { - LOG.error(_name + "/GraphiteMetricsReporterService failed to report metrics", e); + _executor.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + GraphiteMetricsReporterService.this.reportMetrics(); + } catch (Exception e) { + LOG.error(_name + "/GraphiteMetricsReporterService failed to report metrics", + e); + } } - }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS + }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS ); LOG.info("{}/GraphiteMetricsReporterService started", _name); } diff --git a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java index 57dc8028..c61599db 100644 --- a/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/ConsumeServiceTest.java @@ -43,7 +43,7 @@ public class ConsumeServiceTest { private static final String TAGS_NAME = "name"; private static final String METRIC_GROUP_NAME = "commit-availability-service"; /* thread start delay in seconds */ - private static final long THREAD_START_DELAY = 4; + private static final long THREAD_START_DELAY_SECONDS = 4; private static final String TAG_NAME_VALUE = "name"; private static final long MOCK_LAST_COMMITTED_OFFSET = System.currentTimeMillis(); private static final int PARTITION = 2; @@ -83,7 +83,7 @@ public void commitAvailabilityTest() throws Exception { Assert.assertTrue(consumeService.isRunning()); /* in milliseconds */ - long threadStartDelay = 1000 * THREAD_START_DELAY; + long threadStartDelay = TimeUnit.SECONDS.toMillis(THREAD_START_DELAY_SECONDS); /* Thread.sleep safe to do here instead of ScheduledExecutorService * We want to sleep current thread so that consumeService can start running for enough seconds. */ @@ -112,7 +112,7 @@ public void commitLatencyTest() throws Exception { Assert.assertTrue(consumeService.isRunning()); /* in milliseconds */ - long threadStartDelay = TimeUnit.SECONDS.toMillis(THREAD_START_DELAY); + long threadStartDelay = TimeUnit.SECONDS.toMillis(THREAD_START_DELAY_SECONDS); /* Thread.sleep safe to do here instead of ScheduledExecutorService * We want to sleep current thread so that consumeService can start running for enough seconds. */ From 138fe46a06735f19e985dee8f27f4a8e82f572b3 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 22 Mar 2020 15:16:18 -0700 Subject: [PATCH 093/192] Metrics Export Service Config and Properties to include Commit-latency-service -> commit-offset-latency-ms-avg (#227) Append: "kmf.services:type=commit-latency-service,name=:commit-offset-latency-ms-avg", "kmf.services:type=commit-latency-service,name=:commit-offset-latency-ms-max", "kmf.services:type=commit-latency-service,name=:commit-offset-latency-ms-99th", "kmf.services:type=commit-latency-service,name=:commit-offset-latency-ms-999th", "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-9999th" --> for consumer offset commit latency metrics. Remove: "kmf.services:type=commit-availability-service,name=:commit-latency-avg", "kmf.services:type=commit-availability-service,name=:commit-availability-avg", --> commit-latency-avg" and commit-availability-avg are outdated names which is renamed to commit-offset-latency-ms-avg. Signed-off-by: Andrew Choi --- config/kafka-monitor.properties | 7 +++++-- .../java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 6 +++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 4b629ff4..79f8b816 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -81,8 +81,6 @@ "kmf.services:type=produce-service,name=*:produce-availability-avg", "kmf.services:type=consume-service,name=*:consume-availability-avg", "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", - "kmf.services:type=commit-availability-service,name=*:commit-latency-avg", - "kmf.services:type=commit-availability-service,name=*:commit-availability-avg", "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", "kmf.services:type=produce-service,name=*:records-produced-total", "kmf.services:type=consume-service,name=*:records-consumed-total", @@ -94,6 +92,11 @@ "kmf.services:type=produce-service,name=*:produce-error-rate", "kmf.services:type=consume-service,name=*:consume-error-rate", "kmf.services:type=commit-availability-service,name=*:offsets-committed-total", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-avg", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-max", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-99th", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-999th", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-9999th", "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-total" ] } diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index 453f82b4..c0ec1778 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -336,7 +336,11 @@ public static void main(String[] args) throws Exception { "kmf.services:type=commit-availability-service,name=*:offsets-committed-avg", "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-total", "kmf.services:type=commit-availability-service,name=*:failed-commit-offsets-avg", - "kmf.services:type=commit-availability-service,name=*:commit-latency-avg" + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-avg", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-max", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-99th", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-999th", + "kmf.services:type=commit-latency-service,name=*:commit-offset-latency-ms-9999th" ); props.put(DefaultMetricsReporterServiceConfig.REPORT_METRICS_CONFIG, metrics); From 021388ff33792342f5cedd97ef28ffadfdb1c436 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sat, 28 Mar 2020 13:29:31 -0700 Subject: [PATCH 094/192] Remove unneeded Try Catch Block #229 1 - removes unneeded try catch block 2 - _commitAvailabilityMetrics._failedCommitOffsets.record(); is already recorded in the previous code portion. Signed-off-by: Andrew Choi li_andchoi@microsoft.com --- .../linkedin/kmf/services/ConsumeService.java | 49 +++++++++---------- 1 file changed, 22 insertions(+), 27 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 25ce2aad..286a70c3 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -128,37 +128,32 @@ record = _baseConsumer.receive(); } int partition = record.partition(); /* Commit availability and commit latency service */ - try { - /* Call commitAsync, wait for a NON-NULL return value (see https://issues.apache.org/jira/browse/KAFKA-6183) */ - OffsetCommitCallback commitCallback = new OffsetCommitCallback() { - @Override - public void onComplete(Map topicPartitionOffsetAndMetadataMap, Exception kafkaException) { - if (kafkaException != null) { - LOG.error("Exception while trying to perform an asynchronous commit.", kafkaException); - _commitAvailabilityMetrics._failedCommitOffsets.record(); - } else { - _commitAvailabilityMetrics._offsetsCommitted.record(); - _commitLatencyMetrics.recordCommitComplete(); - } + /* Call commitAsync, wait for a NON-NULL return value (see https://issues.apache.org/jira/browse/KAFKA-6183) */ + OffsetCommitCallback commitCallback = new OffsetCommitCallback() { + @Override + public void onComplete(Map topicPartitionOffsetAndMetadataMap, Exception kafkaException) { + if (kafkaException != null) { + LOG.error("Exception while trying to perform an asynchronous commit.", kafkaException); + _commitAvailabilityMetrics._failedCommitOffsets.record(); + } else { + _commitAvailabilityMetrics._offsetsCommitted.record(); + _commitLatencyMetrics.recordCommitComplete(); } - }; + } + }; - /* Current timestamp to perform subtraction*/ - long currTimeMillis = System.currentTimeMillis(); + /* Current timestamp to perform subtraction*/ + long currTimeMillis = System.currentTimeMillis(); - /* 4 seconds consumer offset commit interval. */ - long timeDiffMillis = TimeUnit.SECONDS.toMillis(COMMIT_TIME_INTERVAL); + /* 4 seconds consumer offset commit interval. */ + long timeDiffMillis = TimeUnit.SECONDS.toMillis(COMMIT_TIME_INTERVAL); - if (currTimeMillis - _baseConsumer.lastCommitted() >= timeDiffMillis) { - /* commit the consumer offset asynchronously with a callback. */ - _baseConsumer.commitAsync(commitCallback); - _commitLatencyMetrics.recordCommitStart(); - /* Record the current time for the committed consumer offset */ - _baseConsumer.updateLastCommit(); - } - } catch (Exception exception) { - LOG.error("Exception while trying to perform an asynchronous commit.", exception); - _commitAvailabilityMetrics._failedCommitOffsets.record(); + if (currTimeMillis - _baseConsumer.lastCommitted() >= timeDiffMillis) { + /* commit the consumer offset asynchronously with a callback. */ + _baseConsumer.commitAsync(commitCallback); + _commitLatencyMetrics.recordCommitStart(); + /* Record the current time for the committed consumer offset */ + _baseConsumer.updateLastCommit(); } /* Finished consumer offset commit service. */ From 12a6dd6b98156e27c3cbeee27f2afb4ce0ea3453 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 3 Apr 2020 10:56:48 -0700 Subject: [PATCH 095/192] Add clarification of timestamp in records lost log (#230) Add clarification of timestamp in records lost log since when iterating through logs for debugging, it wasn't clear what the number meant. Add timestamp clarifications. Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 286a70c3..17d85417 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -184,7 +184,7 @@ public void onComplete(Map topicPartitionOffs nextIndexes.put(partition, index + 1); long numLostRecords = index - nextIndex; _sensors._recordsLost.record(numLostRecords); - LOG.info("_recordsLost recorded: Avro record current index: {} at {}. Next index: {}. Lost {} records.", index, currMs, nextIndex, numLostRecords); + LOG.info("_recordsLost recorded: Avro record current index: {} at timestamp {}. Next index: {}. Lost {} records.", index, currMs, nextIndex, numLostRecords); } } /* end of consume() while loop */ From 397a29cea48bcea4313a03bbc53f9b44c7394329 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 6 Apr 2020 09:49:01 -0700 Subject: [PATCH 096/192] javadoc for consume Service (#234) Addition of JavaDoc for Consume Service Signed-off-by: Andrew Choi --- .../com/linkedin/kmf/services/ConsumeService.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 17d85417..b505bb61 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -61,6 +61,21 @@ public class ConsumeService implements Service { private static final String METRIC_GROUP_NAME = "consume-service"; private static Map tags; + /** + * Mainly contains services for three metrics: + * 1 - ConsumeAvailability metrics + * 2 - CommitOffsetAvailability metrics + * 2.1 - commitAvailabilityMetrics records offsets committed upon success. that is, no exception upon callback + * 2.2 - commitAvailabilityMetrics records offsets commit fail upon failure. that is, exception upon callback + * 3 - CommitOffsetLatency metrics + * 3.1 - commitLatencyMetrics records the latency between last successful callback and start of last recorded commit. + * + * @param name Name of the Monitor instance + * @param topicPartitionResult The completable future for topic partition + * @param consumerFactory Consumer Factory object. + * @throws ExecutionException + * @throws InterruptedException + */ public ConsumeService(String name, CompletableFuture topicPartitionResult, ConsumerFactory consumerFactory) From f344ab75021f227fcb6af2d0790c5c2d31aeb10f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 7 Apr 2020 12:46:06 -0700 Subject: [PATCH 097/192] Performance Improvements with Collection.singletonList (#231) Performance Improvements with Collection.singletonList 1 - use Collections.singletonList("kmf.services:*:*")list for performance improvements 2 - Redundant assignments removal. 3 - Use MetricValue casted to double. Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/services/ConsumeMetrics.java | 2 +- .../kmf/services/MultiClusterTopicManagementService.java | 3 +-- .../java/com/linkedin/kmf/services/ProduceService.java | 4 ++-- .../configs/DefaultMetricsReporterServiceConfig.java | 7 +++---- .../configs/GraphiteMetricsReporterServiceConfig.java | 8 +++----- .../configs/KafkaMetricsReporterServiceConfig.java | 8 +++----- .../configs/SignalFxMetricsReporterServiceConfig.java | 6 ++---- .../configs/StatsdMetricsReporterServiceConfig.java | 8 +++----- 8 files changed, 18 insertions(+), 28 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java index d16afb51..f2518827 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java @@ -77,7 +77,7 @@ public class ConsumeMetrics { metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), (config, now) -> { - double recordsConsumedRate = metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsConsumedRate = (double) metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).metricValue(); double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 2df1ec43..7ad9da84 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -314,8 +314,7 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup } private Set getAvailableBrokers() throws ExecutionException, InterruptedException { - Set brokers = new HashSet<>(); - brokers.addAll(_adminClient.describeCluster().nodes().get()); + Set brokers = new HashSet<>(_adminClient.describeCluster().nodes().get()); Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); return brokers; diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index b497d9d5..ed17d88d 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -57,7 +57,7 @@ public class ProduceService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ProduceService.class); private static final String METRIC_GROUP_NAME = "produce-service"; - private static final String[] NONOVERRIDABLE_PROPERTIES = new String[]{ + private static final String[] NON_OVERRIDABLE_PROPERTIES = new String[]{ ProduceServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ProduceServiceConfig.ZOOKEEPER_CONNECT_CONFIG }; @@ -108,7 +108,7 @@ public ProduceService(Map props, String name) throws Exception { _producerPropsOverride = props.containsKey(ProduceServiceConfig.PRODUCER_PROPS_CONFIG) ? (Map) props.get(ProduceServiceConfig.PRODUCER_PROPS_CONFIG) : new HashMap<>(); - for (String property: NONOVERRIDABLE_PROPERTIES) { + for (String property: NON_OVERRIDABLE_PROPERTIES) { if (_producerPropsOverride.containsKey(property)) { throw new ConfigException("Override must not contain " + property + " config."); } diff --git a/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java index 26393d87..f7c3704c 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/DefaultMetricsReporterServiceConfig.java @@ -10,10 +10,10 @@ package com.linkedin.kmf.services.configs; +import java.util.Collections; +import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import java.util.Arrays; -import java.util.Map; public class DefaultMetricsReporterServiceConfig extends AbstractConfig { @@ -27,8 +27,7 @@ public class DefaultMetricsReporterServiceConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, - ConfigDef.Type.LIST, - Arrays.asList("kmf.services:*:*"), + ConfigDef.Type.LIST, Collections.singletonList("kmf.services:*:*"), ConfigDef.Importance.MEDIUM, REPORT_METRICS_DOC) .define(REPORT_INTERVAL_SEC_CONFIG, diff --git a/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java index f78a04c6..4907fef3 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/GraphiteMetricsReporterServiceConfig.java @@ -10,12 +10,11 @@ package com.linkedin.kmf.services.configs; +import java.util.Collections; +import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import java.util.Arrays; -import java.util.Map; - public class GraphiteMetricsReporterServiceConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -36,8 +35,7 @@ public class GraphiteMetricsReporterServiceConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, - ConfigDef.Type.LIST, - Arrays.asList("kmf.services:*:*"), + ConfigDef.Type.LIST, Collections.singletonList("kmf.services:*:*"), ConfigDef.Importance.MEDIUM, REPORT_METRICS_DOC) .define(REPORT_INTERVAL_SEC_CONFIG, diff --git a/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java index c97c6f84..4ceb5123 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/KafkaMetricsReporterServiceConfig.java @@ -10,12 +10,11 @@ package com.linkedin.kmf.services.configs; +import java.util.Collections; +import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import java.util.Arrays; -import java.util.Map; - import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; public class KafkaMetricsReporterServiceConfig extends AbstractConfig { @@ -43,8 +42,7 @@ public class KafkaMetricsReporterServiceConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, - ConfigDef.Type.LIST, - Arrays.asList("kmf.services:*:*"), + ConfigDef.Type.LIST, Collections.singletonList("kmf.services:*:*"), ConfigDef.Importance.MEDIUM, REPORT_METRICS_DOC) .define(REPORT_INTERVAL_SEC_CONFIG, diff --git a/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java index 8781a969..1b5c3138 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/SignalFxMetricsReporterServiceConfig.java @@ -4,9 +4,8 @@ package com.linkedin.kmf.services.configs; -import java.util.Arrays; +import java.util.Collections; import java.util.Map; - import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -34,8 +33,7 @@ public class SignalFxMetricsReporterServiceConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, - ConfigDef.Type.LIST, - Arrays.asList("kmf.services:*:*"), + ConfigDef.Type.LIST, Collections.singletonList("kmf.services:*:*"), ConfigDef.Importance.MEDIUM, REPORT_METRICS_DOC) .define(REPORT_INTERVAL_SEC_CONFIG, diff --git a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java index 60639931..0a431ced 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java @@ -14,12 +14,11 @@ package com.linkedin.kmf.services.configs; +import java.util.Collections; +import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import java.util.Arrays; -import java.util.Map; - public class StatsdMetricsReporterServiceConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -40,8 +39,7 @@ public class StatsdMetricsReporterServiceConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(REPORT_METRICS_CONFIG, - ConfigDef.Type.LIST, - Arrays.asList("kmf.services:*:*"), + ConfigDef.Type.LIST, Collections.singletonList("kmf.services:*:*"), ConfigDef.Importance.MEDIUM, REPORT_METRICS_DOC) .define(REPORT_INTERVAL_SEC_CONFIG, From 27592e049407eaf7b83658e87e8c11ef558b558f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 7 Apr 2020 15:54:21 -0700 Subject: [PATCH 098/192] MetricValue casted to double to avoid deprecated APIs. (#232) * use metric Value casted to double Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java index f2518827..1fe95267 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeMetrics.java @@ -78,8 +78,8 @@ public class ConsumeMetrics { metrics.addMetric(new MetricName("consume-availability-avg", METRIC_GROUP_NAME, "The average consume availability", tags), (config, now) -> { double recordsConsumedRate = (double) metrics.metrics().get(metrics.metricName("records-consumed-rate", METRIC_GROUP_NAME, tags)).metricValue(); - double recordsLostRate = metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).value(); - double recordsDelayedRate = metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).value(); + double recordsLostRate = (double) metrics.metrics().get(metrics.metricName("records-lost-rate", METRIC_GROUP_NAME, tags)).metricValue(); + double recordsDelayedRate = (double) metrics.metrics().get(metrics.metricName("records-delayed-rate", METRIC_GROUP_NAME, tags)).metricValue(); if (new Double(recordsLostRate).isNaN()) recordsLostRate = 0; From 89d05861588a1f116739d8e69b9d938cd1a29954 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 10 Apr 2020 16:29:50 -0700 Subject: [PATCH 099/192] KafkaMonitor constructor creates apps and services for each of the individual clusters (properties) that's passed in. (#238) 1 - Update to use meaningful constructor variables in the class. 2 - Addition of JavaDoc to the same constructor for information Signed-off-by: Andrew Choi li_andchoi@microsoft.com --- .../java/com/linkedin/kmf/KafkaMonitor.java | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/KafkaMonitor.java index 4b763e21..db7c745d 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/KafkaMonitor.java @@ -52,24 +52,32 @@ public class KafkaMonitor { private final ConcurrentMap _services; private final ConcurrentMap _offlineRunnables; private final ScheduledExecutorService _executor; - /** When true start has been called on this instance of Kafka monitor. */ + /** When true start has been called on this instance of Xinfra Monitor. */ private final AtomicBoolean _isRunning = new AtomicBoolean(false); - public KafkaMonitor(Map testProps) throws Exception { + /** + * KafkaMonitor constructor creates apps and services for each of the individual clusters (properties) that's passed in. + * For example, if there are 10 clusters to be monitored, then this Constructor will create 10 * num_apps_per_cluster + * and 10 * num_services_per_cluster. + * @param allClusterProps the properties of ALL kafka clusters for which apps and services need to be appended. + * @throws Exception + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + public KafkaMonitor(Map allClusterProps) throws Exception { _apps = new ConcurrentHashMap<>(); _services = new ConcurrentHashMap<>(); - for (Map.Entry entry : testProps.entrySet()) { - String name = entry.getKey(); - Map props = entry.getValue(); + for (Map.Entry clusterProperty : allClusterProps.entrySet()) { + String name = clusterProperty.getKey(); + Map props = clusterProperty.getValue(); if (!props.containsKey(CLASS_NAME_CONFIG)) throw new IllegalArgumentException(name + " is not configured with " + CLASS_NAME_CONFIG); String className = (String) props.get(CLASS_NAME_CONFIG); Class aClass = Class.forName(className); if (App.class.isAssignableFrom(aClass)) { - App test = (App) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); - _apps.put(name, test); + App clusterApp = (App) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); + _apps.put(name, clusterApp); } else if (Service.class.isAssignableFrom(aClass)) { Constructor[] constructors = Class.forName(className).getConstructors(); if (this.constructorContainsFuture(constructors)) { @@ -124,7 +132,7 @@ public synchronized void start() throws Exception { try { checkHealth(); } catch (Exception e) { - LOG.error("Failed to check health of tests and services", e); + LOG.error("Failed to check health of apps and services", e); } }, initialDelaySecond, periodSecond, TimeUnit.SECONDS ); @@ -155,19 +163,20 @@ public synchronized void stop() { return; } _executor.shutdownNow(); - for (App test: _apps.values()) - test.stop(); + for (App app: _apps.values()) + app.stop(); for (Service service: _services.values()) service.stop(); } public void awaitShutdown() { - for (App test: _apps.values()) - test.awaitShutdown(); + for (App app: _apps.values()) + app.awaitShutdown(); for (Service service: _services.values()) service.awaitShutdown(); } + @SuppressWarnings("rawtypes") public static void main(String[] args) throws Exception { if (args.length <= 0) { LOG.info("USAGE: java [options] " + KafkaMonitor.class.getName() + " config/kafka-monitor.properties"); @@ -187,7 +196,7 @@ public static void main(String[] args) throws Exception { Map props = new ObjectMapper().readValue(buffer.toString(), Map.class); KafkaMonitor kafkaMonitor = new KafkaMonitor(props); kafkaMonitor.start(); - LOG.info("KafkaMonitor started."); + LOG.info("Xinfra Monitor (KafkaMonitor) started."); kafkaMonitor.awaitShutdown(); } From e698b789eabb1d1fa3e9f5aef9cb53ff7cca568f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 10 Apr 2020 19:39:37 -0700 Subject: [PATCH 100/192] PreferredLeaderElections (PLE) Kafka Clients 2.4.0 (#237) _adminClient.electPreferredLeaders(...) is deprecated. Update the class method to use the new API for PreferredLeaderElections (PLE) in Kafka Clients 2.4.0. Signed-off-by: Andrew Choi --- .../MultiClusterTopicManagementService.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 7ad9da84..ec5cf2c8 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -37,10 +37,12 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.CreateTopicsResult; -import org.apache.kafka.clients.admin.ElectPreferredLeadersResult; +import org.apache.kafka.clients.admin.ElectLeadersOptions; +import org.apache.kafka.clients.admin.ElectLeadersResult; import org.apache.kafka.clients.admin.NewPartitions; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; @@ -69,6 +71,7 @@ * - Make sure the number of partitions of the monitor topic is same across all monitored clusters. * */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class MultiClusterTopicManagementService implements Service { private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); private static final String METRIC_GROUP_NAME = "topic-management-service"; @@ -415,9 +418,13 @@ private void triggerPreferredLeaderElection(List partitionIn for (TopicPartitionInfo javaPartitionInfo : partitionInfoList) { partitions.add(new TopicPartition(partitionTopic, javaPartitionInfo.partition())); } - ElectPreferredLeadersResult electPreferredLeadersResult = _adminClient.electPreferredLeaders(partitions); - LOG.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electPreferredLeadersResult.all().get()); + ElectLeadersOptions newOptions = new ElectLeadersOptions(); + ElectionType electionType = ElectionType.PREFERRED; + Set topicPartitions = new HashSet<>(partitions); + ElectLeadersResult electLeadersResult = _adminClient.electLeaders(electionType, topicPartitions, newOptions); + + LOG.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electLeadersResult.all().get()); } private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { From b41834250d1c29560e7f29c18481b48e10934cc7 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 15 Apr 2020 11:17:03 -0700 Subject: [PATCH 101/192] Fix NoSuchFieldError DEFAULT_SSL_PRINCIPAL_MAPPING_RULES (#239) 1 - Minor codestyle spacing 2 - Add SuppressWarnings for "rawtypes" 3 - Add no inspections on rawtypes 4 - Fix NoSuchFieldError DEFAULT_SSL_PRINCIPAL_MAPPING_RULES when KafkaMetricsReporterService is executed by removing unnecessary elements in the log. Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/common/Utils.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index bf35a0eb..5fcbf55e 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -25,7 +25,6 @@ import javax.management.MBeanInfo; import javax.management.MBeanServer; import javax.management.ObjectName; -import kafka.server.KafkaConfig; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; @@ -38,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** * Kafka Monitoring utilities. */ @@ -93,8 +93,10 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, int partitionCount = Math.max((int) Math.ceil(brokerCount * partitionToBrokerRatio), minPartitionNum); try { NewTopic newTopic = new NewTopic(topic, partitionCount, replicationFactor); + //noinspection rawtypes newTopic.configs((Map) topicConfig); - List topics = new ArrayList(); + + List topics = new ArrayList<>(); topics.add(newTopic); adminClient.createTopics(topics); } catch (TopicExistsException e) { @@ -102,12 +104,12 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, LOG.debug("Monitoring topic " + topic + " already exists in the cluster.", e); return getPartitionNumForTopic(adminClient, topic); } - LOG.info("Created monitoring topic " + topic + " in cluster with " + partitionCount + " partitions, min ISR of " - + topicConfig.get(KafkaConfig.MinInSyncReplicasProp()) + " and replication factor of " + replicationFactor + "."); + LOG.info("Created monitoring topic {} in cluster with {} partitions and replication factor of {}.", topic, + partitionCount, replicationFactor); return partitionCount; } finally { - LOG.info("Completed the topic creation if it doesn't exist."); + LOG.info("Completed the topic creation if it doesn't exist for {}", topic); } } @@ -123,7 +125,7 @@ private static int getBrokerCount(AdminClient adminClient) throws ExecutionExcep * @param topic topic this message is sent to * @param idx index is consecutive numbers used by KafkaMonitor to determine duplicate or lost messages * @param msgSize size of the message - * @return string that encodes the above fields + * @return string that encodes the above fields */ public static String jsonFromFields(String topic, long idx, long timestamp, String producerId, int msgSize) { GenericRecord record = new GenericData.Record(DefaultTopicSchema.MESSAGE_V0); @@ -138,7 +140,7 @@ public static String jsonFromFields(String topic, long idx, long timestamp, Stri /** * @param message kafka message in the string format - * @return GenericRecord that is de-serialized from kafka message w.r.t. expected schema. + * @return GenericRecord that is de-serialized from kafka message w.r.t. expected schema. */ public static GenericRecord genericRecordFromJson(String message) { GenericRecord record = new GenericData.Record(DefaultTopicSchema.MESSAGE_V0); @@ -146,7 +148,8 @@ public static GenericRecord genericRecordFromJson(String message) { record.put(DefaultTopicSchema.TOPIC_FIELD.name(), jsonObject.getString(DefaultTopicSchema.TOPIC_FIELD.name())); record.put(DefaultTopicSchema.INDEX_FIELD.name(), jsonObject.getLong(DefaultTopicSchema.INDEX_FIELD.name())); record.put(DefaultTopicSchema.TIME_FIELD.name(), jsonObject.getLong(DefaultTopicSchema.TIME_FIELD.name())); - record.put(DefaultTopicSchema.PRODUCER_ID_FIELD.name(), jsonObject.getString(DefaultTopicSchema.PRODUCER_ID_FIELD.name())); + record.put(DefaultTopicSchema.PRODUCER_ID_FIELD.name(), + jsonObject.getString(DefaultTopicSchema.PRODUCER_ID_FIELD.name())); record.put(DefaultTopicSchema.CONTENT_FIELD.name(), jsonObject.getString(DefaultTopicSchema.CONTENT_FIELD.name())); return record; } @@ -170,11 +173,12 @@ public static List getMBeanAttributeValues(String mbeanExpr MBeanServer server = ManagementFactory.getPlatformMBeanServer(); try { Set mbeanNames = server.queryNames(new ObjectName(mbeanExpr), null); - for (ObjectName mbeanName: mbeanNames) { + for (ObjectName mbeanName : mbeanNames) { MBeanInfo mBeanInfo = server.getMBeanInfo(mbeanName); MBeanAttributeInfo[] attributeInfos = mBeanInfo.getAttributes(); - for (MBeanAttributeInfo attributeInfo: attributeInfos) { - if (attributeInfo.getName().equals(attributeExpr) || attributeExpr.length() == 0 || attributeExpr.equals("*")) { + for (MBeanAttributeInfo attributeInfo : attributeInfos) { + if (attributeInfo.getName().equals(attributeExpr) || attributeExpr.length() == 0 || attributeExpr.equals( + "*")) { double value = (Double) server.getAttribute(mbeanName, attributeInfo.getName()); values.add(new MbeanAttributeValue(mbeanName.getCanonicalName(), attributeInfo.getName(), value)); } From f681dc43334b497da71157925760904ab0ebffd6 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 16 Apr 2020 21:13:32 -0700 Subject: [PATCH 102/192] change LOG level to debug for commit offsets already in progress (#241) Offset commit already in progress is not a critical error that should kill the process. Some commit tasks (Futures threads) take longer to complete and due to the nature of async commits, some commits take longer. Thus, changing level of Logging to debug is sufficient here. Signed-off-by: Andrew Choi li_andchoi@microsoft.com --- .../java/com/linkedin/kmf/services/CommitLatencyMetrics.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java index 33c22385..d0ee18d0 100644 --- a/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java +++ b/src/main/java/com/linkedin/kmf/services/CommitLatencyMetrics.java @@ -68,7 +68,7 @@ public void recordCommitStart() throws Exception { _inProgressCommit = true; } else { // inProgressCommit is already set to TRUE; - throw new Exception("Offset commit is already in progress."); + LOG.debug("Offset commit is already in progress."); } } @@ -83,7 +83,7 @@ public void recordCommitComplete() { _inProgressCommit = false; } else { // inProgressCommit is already set to FALSE; - LOG.error("Offset commit is not in progress. CommitLatencyMetrics shouldn't completing a record commit here."); + LOG.debug("Offset commit is not in progress. CommitLatencyMetrics shouldn't completing a record commit here."); } } From cd35295fd47db31fe26b21112a3ecf1d3b977f9f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 16 Apr 2020 22:49:36 -0700 Subject: [PATCH 103/192] LOG lines have wrong number of arguments (#242) LOG errors lines have wrong number of arguments in ProduceService -> Fix this Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/services/ProduceService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index ed17d88d..0094b280 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -395,9 +395,9 @@ public void run() { initializeStateForPartitions(currentPartitionNum); LOG.info("New partitions added to monitoring."); } catch (InterruptedException e) { - LOG.error("InterruptedException occurred {}.", e); + LOG.error("InterruptedException occurred.", e); } catch (ExecutionException e) { - LOG.error("ExecutionException occurred {}.", e); + LOG.error("ExecutionException occurred.", e); } } } From b6d5802cca8e139e5cf8d1207e3933c8cb272c38 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 14 May 2020 21:39:40 -0700 Subject: [PATCH 104/192] addition of CircleCI Signed-off-by: Andrew Choi --- .circleci/config.yml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 .circleci/config.yml diff --git a/.circleci/config.yml b/.circleci/config.yml new file mode 100644 index 00000000..996268f9 --- /dev/null +++ b/.circleci/config.yml @@ -0,0 +1,8 @@ +version: 2.1 +jobs: + build: + docker: + - image: circleci/node:4.8.2 # the primary container, where your job's commands are run + steps: + - checkout # check out the code in the project directory + - run: echo "Kafka Monitor CircleCI" From d10f7df4f08a914a00f42c6ae0a2686f9ec37d31 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 15 May 2020 14:18:58 -0700 Subject: [PATCH 105/192] Get topicPartitionsWithRetry and a minor spelling error in services/MultiClusterTopicManagementService (#243) 1 - minor spelling error in services/MultiClusterTopicManagementService `satisifies` 2 - Get topicPartitionsWithRetry after Creation of Topic 3 - LOGGER renaming 4 - Mark CircleCI as TODO --- .circleci/config.yml | 19 ++++-- .../MultiClusterTopicManagementService.java | 64 ++++++++++--------- 2 files changed, 47 insertions(+), 36 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 996268f9..85f9a2a1 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1,8 +1,13 @@ -version: 2.1 -jobs: - build: - docker: - - image: circleci/node:4.8.2 # the primary container, where your job's commands are run +version: 2 # use CircleCI 2.0 + +jobs: # a collection of steps + build: # runs not using Workflows must have a `build` job as entry point + + docker: # run the steps with Docker + - image: circleci/openjdk:8-jdk-stretch # with this image as the primary container; this is where all `steps` will run. + steps: - - checkout # check out the code in the project directory - - run: echo "Kafka Monitor CircleCI" + # TODO: populate the necessary steps for CircleCI + - run: echo "Kafka Monitor CircleCI on GitHub" + + # See https://circleci.com/docs/2.0/deployment-integrations/ for deploy examples \ No newline at end of file diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index ec5cf2c8..4dd0c9e0 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -73,7 +73,7 @@ */ @SuppressWarnings({"rawtypes", "unchecked"}) public class MultiClusterTopicManagementService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); + private static final Logger LOGGER = LoggerFactory.getLogger(MultiClusterTopicManagementService.class); private static final String METRIC_GROUP_NAME = "topic-management-service"; private final CompletableFuture _topicPartitionResult = new CompletableFuture<>(); private final AtomicBoolean _isRunning = new AtomicBoolean(false); @@ -126,7 +126,7 @@ public synchronized void start() { Runnable pleRunnable = new PreferredLeaderElectionRunnable(); _executor.scheduleWithFixedDelay(pleRunnable, _preferredLeaderElectionIntervalMs, _preferredLeaderElectionIntervalMs, TimeUnit.MILLISECONDS); - LOG.info("{}/MultiClusterTopicManagementService started.", _serviceName); + LOGGER.info("{}/MultiClusterTopicManagementService started.", _serviceName); } } @@ -134,7 +134,7 @@ public synchronized void start() { public synchronized void stop() { if (_isRunning.compareAndSet(true, false)) { _executor.shutdown(); - LOG.info("{}/MultiClusterTopicManagementService stopped.", _serviceName); + LOGGER.info("{}/MultiClusterTopicManagementService stopped.", _serviceName); } } @@ -148,9 +148,9 @@ public void awaitShutdown() { try { _executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.info("Thread interrupted when waiting for {}/MultiClusterTopicManagementService to shutdown", _serviceName); + LOGGER.info("Thread interrupted when waiting for {}/MultiClusterTopicManagementService to shutdown", _serviceName); } - LOG.info("{}/MultiClusterTopicManagementService shutdown completed", _serviceName); + LOGGER.info("{}/MultiClusterTopicManagementService shutdown completed", _serviceName); } @@ -165,7 +165,7 @@ public void run() { } /* - * The partition number of the monitor topics should be the minimum partition number that satisifies the following conditions: + * The partition number of the monitor topics should be the minimum partition number that satisfies the following conditions: * - partition number of the monitor topics across all monitored clusters should be the same * - partitionNum / brokerNum >= user-configured partitionsToBrokersRatio. * - partitionNum >= user-configured minPartitionNum @@ -185,13 +185,13 @@ public void run() { try { helper.maybeReassignPartitionAndElectLeader(); } catch (IOException | KafkaException e) { - LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); + LOGGER.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); } } } catch (Throwable t) { // Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime // and such error is not caught by compilation - LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); + LOGGER.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); stop(); } } @@ -211,18 +211,20 @@ public void run() { try { helper.maybeElectLeader(); } catch (IOException | KafkaException e) { - LOG.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); + LOGGER.warn(_serviceName + "/MultiClusterTopicManagementService will retry later in cluster " + clusterName, e); } } } catch (Throwable t) { /* Need to catch throwable because there is scala API that can throw NoSuchMethodError in runtime and such error is not caught by compilation. */ - LOG.error(_serviceName + "/MultiClusterTopicManagementService will stop due to error.", t); + LOGGER.error(_serviceName + + "/MultiClusterTopicManagementService/PreferredLeaderElectionRunnable will stop due to an error.", t); stop(); } } } + @SuppressWarnings("FieldCanBeLocal") static class TopicManagementHelper { private final boolean _topicCreationEnabled; private final String _topic; @@ -233,9 +235,9 @@ static class TopicManagementHelper { private final TopicFactory _topicFactory; private final Properties _topicProperties; private boolean _preferredLeaderElectionRequested; - private int _requestTimeoutMs; - private List _bootstrapServers; - private final AdminClient _adminClient; + private final int _requestTimeoutMs; + private final List _bootstrapServers; + AdminClient _adminClient; @SuppressWarnings("unchecked") @@ -263,7 +265,7 @@ static class TopicManagementHelper { _topicFactory = (TopicFactory) Class.forName(topicFactoryClassName).getConstructor(Map.class).newInstance(topicFactoryConfig); _adminClient = constructAdminClient(props); - LOG.info("{} configs: {}", _adminClient.getClass().getSimpleName(), props); + LOGGER.info("{} configs: {}", _adminClient.getClass().getSimpleName(), props); } @SuppressWarnings("unchecked") @@ -274,7 +276,10 @@ void maybeCreateTopic() throws Exception { NewTopic newTopic = new NewTopic(_topic, numPartitions, (short) _replicationFactor); newTopic.configs((Map) _topicProperties); CreateTopicsResult createTopicsResult = _adminClient.createTopics(Collections.singletonList(newTopic)); - LOG.info("CreateTopicsResult: {}.", createTopicsResult.values()); + + // waits for this topic creation future to complete, and then returns its result. + createTopicsResult.values().get(_topic).get(); + LOGGER.info("CreateTopicsResult: {}.", createTopicsResult.values()); } } @@ -288,13 +293,14 @@ int minPartitionNum() throws InterruptedException, ExecutionException { } void maybeAddPartitions(int minPartitionNum) throws ExecutionException, InterruptedException { - Collection topicNames = _adminClient.listTopics().names().get(); - Map> kafkaFutureMap = _adminClient.describeTopics(topicNames).values(); + Map> kafkaFutureMap = + _adminClient.describeTopics(Collections.singleton(_topic)).values(); KafkaFuture topicDescriptions = kafkaFutureMap.get(_topic); List partitions = topicDescriptions.get().partitions(); + int partitionNum = partitions.size(); if (partitionNum < minPartitionNum) { - LOG.info("{} will increase partition of the topic {} in the cluster from {}" + LOGGER.info("{} will increase partition of the topic {} in the cluster from {}" + " to {}.", this.getClass().toString(), _topic, partitionNum, minPartitionNum); Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); List> replicaAssignment = new ArrayList<>(new ArrayList<>()); @@ -339,13 +345,13 @@ void maybeReassignPartitionAndElectLeader() throws Exception { int expectedReplicationFactor = Math.max(currentReplicationFactor, _replicationFactor); if (_replicationFactor < currentReplicationFactor) - LOG.debug( + LOGGER.debug( "Configured replication factor {} is smaller than the current replication factor {} of the topic {} in cluster.", _replicationFactor, currentReplicationFactor, _topic); if (expectedReplicationFactor > currentReplicationFactor && !zkClient .reassignPartitionsInProgress()) { - LOG.info( + LOGGER.info( "MultiClusterTopicManagementService will increase the replication factor of the topic {} in cluster" + "from {} to {}", _topic, currentReplicationFactor, expectedReplicationFactor); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), @@ -362,7 +368,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { expectedProperties.put(key, _topicProperties.get(key)); if (!currentProperties.equals(expectedProperties)) { - LOG.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + LOGGER.info("MultiClusterTopicManagementService will overwrite properties of the topic {} " + "in cluster from {} to {}.", _topic, currentProperties, expectedProperties); zkClient.setOrCreateEntityConfigs(ConfigType.Topic(), _topic, expectedProperties); } @@ -370,7 +376,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { if (partitionInfoList.size() >= brokers.size() && someBrokerNotPreferredLeader(partitionInfoList, brokers) && !zkClient .reassignPartitionsInProgress()) { - LOG.info("{} will reassign partitions of the topic {} in cluster.", + LOGGER.info("{} will reassign partitions of the topic {} in cluster.", this.getClass().toString(), _topic); reassignPartitions(zkClient, brokers, _topic, partitionInfoList.size(), expectedReplicationFactor); @@ -380,7 +386,7 @@ void maybeReassignPartitionAndElectLeader() throws Exception { if (partitionInfoList.size() >= brokers.size() && someBrokerNotElectedLeader(partitionInfoList, brokers)) { if (!partitionReassigned || !zkClient.reassignPartitionsInProgress()) { - LOG.info( + LOGGER.info( "MultiClusterTopicManagementService will trigger preferred leader election for the topic {} in " + "cluster.", _topic ); @@ -403,7 +409,7 @@ void maybeElectLeader() throws Exception { if (!zkClient.reassignPartitionsInProgress()) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); - LOG.info( + LOGGER.info( "MultiClusterTopicManagementService will trigger requested preferred leader election for the" + " topic {} in cluster.", _topic); triggerPreferredLeaderElection(partitionInfoList, _topic); @@ -424,7 +430,7 @@ private void triggerPreferredLeaderElection(List partitionIn Set topicPartitions = new HashSet<>(partitions); ElectLeadersResult electLeadersResult = _adminClient.electLeaders(electionType, topicPartitions, newOptions); - LOG.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electLeadersResult.all().get()); + LOGGER.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electLeadersResult.all().get()); } private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { @@ -448,9 +454,9 @@ private static void reassignPartitions(KafkaZkClient zkClient, Collection String currentAssignmentJson = formatAsReassignmentJson(topic, currentAssignment); String newAssignmentJson = formatAsReassignmentJson(topic, assignedReplicas); - LOG.info("Reassign partitions for topic " + topic); - LOG.info("Current partition replica assignment " + currentAssignmentJson); - LOG.info("New partition replica assignment " + newAssignmentJson); + LOGGER.info("Reassign partitions for topic " + topic); + LOGGER.info("Current partition replica assignment " + currentAssignmentJson); + LOGGER.info("New partition replica assignment " + newAssignmentJson); zkClient.createPartitionReassignment(newAssignment); } @@ -461,7 +467,7 @@ static int getReplicationFactor(List partitionInfoList) { int replicationFactor = partitionInfoList.get(0).replicas().size(); for (TopicPartitionInfo partitionInfo : partitionInfoList) { if (replicationFactor != partitionInfo.replicas().size()) { - LOG.warn("Partitions of the topic have different replication factor."); + LOGGER.warn("Partitions of the topic have different replication factor."); return -1; } } From dd5bd65fa24ac1632e83406acbc6b88ec8eba218 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 18 May 2020 11:06:03 -0700 Subject: [PATCH 106/192] Addition of KMF/services/MultiClusterTopicManagementServiceTest.java (#244) Addition of KMF/services/MultiClusterTopicManagementServiceTest.java Signed-off-by: Andrew Choi --- .../MultiClusterTopicManagementService.java | 20 ++- ...ultiClusterTopicManagementServiceTest.java | 135 ++++++++++++++++++ 2 files changed, 153 insertions(+), 2 deletions(-) create mode 100644 src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 4dd0c9e0..68a7060f 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -226,8 +226,6 @@ public void run() { @SuppressWarnings("FieldCanBeLocal") static class TopicManagementHelper { - private final boolean _topicCreationEnabled; - private final String _topic; private final String _zkConnect; private final int _replicationFactor; private final double _minPartitionsToBrokersRatio; @@ -237,7 +235,11 @@ static class TopicManagementHelper { private boolean _preferredLeaderElectionRequested; private final int _requestTimeoutMs; private final List _bootstrapServers; + + // package private for unit testing + boolean _topicCreationEnabled; AdminClient _adminClient; + String _topic; @SuppressWarnings("unchecked") @@ -322,6 +324,20 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup } } + /** + * Exposed package-private access for testing. Get the total number of partitions for a Kafka topic. + * @return total number of topic partitions + * @throws InterruptedException when a thread is waiting, sleeping and the thread is interrupted, either before / during the activity. + * @throws ExecutionException when attempting to retrieve the result of a task that aborted by throwing an exception. + */ + int numPartitions() throws InterruptedException, ExecutionException { + + // TODO (andrewchoi5): connect this to unit testing method for testing maybeAddPartitions! + + return _adminClient.describeTopics(Collections.singleton(_topic)).values().get(_topic).get().partitions().size(); + } + + private Set getAvailableBrokers() throws ExecutionException, InterruptedException { Set brokers = new HashSet<>(_adminClient.describeCluster().nodes().get()); Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); diff --git a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java new file mode 100644 index 00000000..7aaaf8d3 --- /dev/null +++ b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java @@ -0,0 +1,135 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.CreateTopicsResult; +import org.apache.kafka.clients.admin.DescribeClusterResult; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.Node; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +/** + * Testing methods for the Xinfra Monitor class of MultiClusterTopicManagementService. + */ +@SuppressWarnings("unchecked") +@Test +public class MultiClusterTopicManagementServiceTest { + private static final String SERVICE_TEST_TOPIC = "xinfra-monitor-Multi-Cluster-Topic-Management-Service-Test-topic"; + private static Set nodeSet; + private MultiClusterTopicManagementService.TopicManagementHelper _topicManagementHelper; + private CreateTopicsResult _createTopicsResult; + private Map> _kafkaFutureMap; + private KafkaFuture _kafkaFuture; + + @BeforeMethod + private void startTest() { + _createTopicsResult = Mockito.mock(CreateTopicsResult.class); + _kafkaFutureMap = Mockito.mock(Map.class); + _kafkaFuture = Mockito.mock(KafkaFuture.class); + + nodeSet = new HashSet<>(); + nodeSet.add(new Node(1, "host-1", 2132)); + nodeSet.add(new Node(2, "host-2", 2133)); + nodeSet.add(new Node(3, "host-3", 2134)); + nodeSet.add(new Node(4, "host-4", 2135)); + nodeSet.add(new Node(5, "host-5", 2136)); + + _topicManagementHelper = Mockito.mock(MultiClusterTopicManagementService.TopicManagementHelper.class); + _topicManagementHelper._topic = SERVICE_TEST_TOPIC; + _topicManagementHelper._adminClient = Mockito.mock(AdminClient.class); + _topicManagementHelper._topicCreationEnabled = true; + } + + @AfterMethod + private void finishTest() { + System.out.println("Finished " + this.getClass().getCanonicalName().toLowerCase() + "."); + } + + @Test + protected void MultiClusterTopicManagementServiceTopicCreationTest() throws Exception { + + Mockito.doCallRealMethod().when(_topicManagementHelper).maybeCreateTopic(); + + Mockito.when(_topicManagementHelper._adminClient.describeCluster()) + .thenReturn(Mockito.mock(DescribeClusterResult.class)); + Mockito.when(_topicManagementHelper._adminClient.describeCluster().nodes()) + .thenReturn(Mockito.mock(KafkaFuture.class)); + Mockito.when(_topicManagementHelper._adminClient.describeCluster().nodes().get()).thenReturn(nodeSet); + + Mockito.when(_topicManagementHelper._adminClient.createTopics(Mockito.anyCollection())) + .thenReturn(_createTopicsResult); + Mockito.when(_topicManagementHelper._adminClient.createTopics(Mockito.anyCollection()).values()) + .thenReturn(_kafkaFutureMap); + Mockito.when( + _topicManagementHelper._adminClient.createTopics(Mockito.anyCollection()).values().get(SERVICE_TEST_TOPIC)) + .thenReturn(_kafkaFuture); + + Answer createKafkaTopicFutureAnswer = new Answer() { + /** + * @param invocation the invocation on the mocked TopicManagementHelper. + * @return NULL value. + * @throws Throwable the throwable to be thrown when Exception occurs. + */ + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Mockito.when(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC))) + .thenReturn(Mockito.mock(DescribeTopicsResult.class)); + Mockito.when( + _topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)).values()) + .thenReturn(Mockito.mock(Map.class)); + Mockito.when(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)) + .values() + .get(SERVICE_TEST_TOPIC)).thenReturn(Mockito.mock(KafkaFuture.class)); + Mockito.when(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)) + .values() + .get(SERVICE_TEST_TOPIC) + .get()).thenReturn(Mockito.mock(TopicDescription.class)); + Mockito.when(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)) + .values() + .get(SERVICE_TEST_TOPIC) + .get() + .name()).thenReturn(SERVICE_TEST_TOPIC); + return null; + } + }; + + Mockito.when(_topicManagementHelper._adminClient.createTopics(Mockito.anyCollection()) + .values() + .get(SERVICE_TEST_TOPIC) + .get()).thenAnswer(createKafkaTopicFutureAnswer); + + _topicManagementHelper.maybeCreateTopic(); + + Assert.assertNotNull(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)) + .values() + .get(SERVICE_TEST_TOPIC) + .get()); + Assert.assertEquals(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC)) + .values() + .get(SERVICE_TEST_TOPIC) + .get() + .name(), SERVICE_TEST_TOPIC); + } +} From a502ed66d6a4429001a27a9ad98f683404cf19df Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 26 May 2020 16:10:32 -0700 Subject: [PATCH 107/192] Add topic log in consume produce service (#247) Add topic log in consume produce service (#247) Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/services/ConsumeService.java | 2 +- src/main/java/com/linkedin/kmf/services/ProduceService.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index b505bb61..fed2c6d0 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -230,7 +230,7 @@ public synchronized void start() { try { topicDescription = topicDescriptionKafkaFuture.get(); } catch (InterruptedException | ExecutionException e) { - LOG.error("Exception occurred while getting the topicDescriptionKafkaFuture", e); + LOG.error("Exception occurred while getting the topicDescriptionKafkaFuture for topic: {}", _topic, e); } double partitionCount = topicDescription.partitions().size(); topicPartitionCount.add( diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 0094b280..23d71326 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -171,7 +171,7 @@ public synchronized void start() { _handleNewPartitionsExecutor.scheduleWithFixedDelay(new NewPartitionHandler(), 1, 30, TimeUnit.SECONDS); LOG.info("{}/ProduceService started", _name); } catch (InterruptedException | UnknownTopicOrPartitionException | ExecutionException e) { - LOG.error("Exception occurred while starting produce service: ", e); + LOG.error("Exception occurred while starting produce service for topic: {}", _topic, e); } } } From ffa7d591ffd8664a599e52cde6006236039ce933 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 26 May 2020 16:31:25 -0700 Subject: [PATCH 108/192] MulticlustertopicmanagementService should block until topic creation is fully complete (#246) * Xinfra Monitor MulticlustertopicmanagementService should block until topic creation is fully complete. Signed-off-by: Andrew Choi * wip Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/apps/SingleClusterMonitor.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java index c0ec1778..de88d2f5 100644 --- a/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java +++ b/src/main/java/com/linkedin/kmf/apps/SingleClusterMonitor.java @@ -62,12 +62,17 @@ public SingleClusterMonitor(Map props, String name) throws Excep _name = name; _topicManagementService = new TopicManagementService(props, name); CompletableFuture topicPartitionResult = _topicManagementService.topicPartitionResult(); + + // block on the MultiClusterTopicManagementService to complete. + topicPartitionResult.get(); + _produceService = new ProduceService(props, name); _consumeService = new ConsumeService(name, topicPartitionResult, consumerFactory); _allServices = new ArrayList<>(SERVICES_INITIAL_CAPACITY); _allServices.add(_topicManagementService); _allServices.add(_produceService); _allServices.add(_consumeService); + } @Override From 2ae902a05a4c19f7520c85d604edb6ff5aaaddf1 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 May 2020 10:23:50 -0700 Subject: [PATCH 109/192] Use createTopicIfNotExist instead of AdminClient (#248) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Use createTopicIfNotExist instead of AdminClient connects the _topicFactory interface’s createTopicIfNotExist. 🙏🏻 Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/common/Utils.java | 8 +++++++- .../services/MultiClusterTopicManagementService.java | 10 +++------- .../MultiClusterTopicManagementServiceTest.java | 9 +++++---- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 5fcbf55e..0d2c2fd5 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -31,6 +31,7 @@ import org.apache.avro.io.Encoder; import org.apache.avro.io.JsonEncoder; 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.common.errors.TopicExistsException; import org.json.JSONObject; @@ -98,7 +99,12 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, List topics = new ArrayList<>(); topics.add(newTopic); - adminClient.createTopics(topics); + CreateTopicsResult result = adminClient.createTopics(topics); + + // waits for this topic creation future to complete, and then returns its result. + result.values().get(topic).get(); + LOG.info("CreateTopicsResult: {}.", result.values()); + } catch (TopicExistsException e) { /* There is a race condition with the consumer. */ LOG.debug("Monitoring topic " + topic + " already exists in the cluster.", e); diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 68a7060f..bc6f923c 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -36,7 +36,6 @@ import kafka.zk.KafkaZkClient; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.ElectLeadersOptions; import org.apache.kafka.clients.admin.ElectLeadersResult; import org.apache.kafka.clients.admin.NewPartitions; @@ -230,7 +229,6 @@ static class TopicManagementHelper { private final int _replicationFactor; private final double _minPartitionsToBrokersRatio; private final int _minPartitionNum; - private final TopicFactory _topicFactory; private final Properties _topicProperties; private boolean _preferredLeaderElectionRequested; private final int _requestTimeoutMs; @@ -240,6 +238,7 @@ static class TopicManagementHelper { boolean _topicCreationEnabled; AdminClient _adminClient; String _topic; + TopicFactory _topicFactory; @SuppressWarnings("unchecked") @@ -277,11 +276,8 @@ void maybeCreateTopic() throws Exception { int numPartitions = Math.max((int) Math.ceil(brokerCount * _minPartitionsToBrokersRatio), minPartitionNum()); NewTopic newTopic = new NewTopic(_topic, numPartitions, (short) _replicationFactor); newTopic.configs((Map) _topicProperties); - CreateTopicsResult createTopicsResult = _adminClient.createTopics(Collections.singletonList(newTopic)); - - // waits for this topic creation future to complete, and then returns its result. - createTopicsResult.values().get(_topic).get(); - LOGGER.info("CreateTopicsResult: {}.", createTopicsResult.values()); + _topicFactory.createTopicIfNotExist(_topic, (short) _replicationFactor, _minPartitionsToBrokersRatio, + _topicProperties, _adminClient); } } diff --git a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java index 7aaaf8d3..da172d06 100644 --- a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java @@ -10,6 +10,7 @@ package com.linkedin.kmf.services; +import com.linkedin.kmf.topicfactory.TopicFactory; import java.util.Collections; import java.util.HashSet; import java.util.Map; @@ -59,6 +60,7 @@ private void startTest() { _topicManagementHelper = Mockito.mock(MultiClusterTopicManagementService.TopicManagementHelper.class); _topicManagementHelper._topic = SERVICE_TEST_TOPIC; _topicManagementHelper._adminClient = Mockito.mock(AdminClient.class); + _topicManagementHelper._topicFactory = Mockito.mock(TopicFactory.class); _topicManagementHelper._topicCreationEnabled = true; } @@ -94,6 +96,7 @@ protected void MultiClusterTopicManagementServiceTopicCreationTest() throws Exce */ @Override public Void answer(InvocationOnMock invocation) throws Throwable { + Mockito.when(_topicManagementHelper._adminClient.describeTopics(Collections.singleton(SERVICE_TEST_TOPIC))) .thenReturn(Mockito.mock(DescribeTopicsResult.class)); Mockito.when( @@ -115,10 +118,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }; - Mockito.when(_topicManagementHelper._adminClient.createTopics(Mockito.anyCollection()) - .values() - .get(SERVICE_TEST_TOPIC) - .get()).thenAnswer(createKafkaTopicFutureAnswer); + Mockito.when(_topicManagementHelper._topicFactory.createTopicIfNotExist(Mockito.anyString(), Mockito.anyShort(), + Mockito.anyDouble(), Mockito.any(), Mockito.any())).thenAnswer(createKafkaTopicFutureAnswer); _topicManagementHelper.maybeCreateTopic(); From c63d44123d09ee6698106fa4dfe002e364fbc1ec Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 May 2020 16:29:30 -0700 Subject: [PATCH 110/192] .gitignore for random unit tests (#250) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit .gitignore for random unit tests (#250)🙏🏻 Signed-off-by: Andrew Choi --- .gitignore | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.gitignore b/.gitignore index 848e31f0..a5b7096f 100644 --- a/.gitignore +++ b/.gitignore @@ -8,3 +8,7 @@ logs/ .settings/ + + + +src/test/java/com/linkedin/kmf/RandomTests.java From 7417f2a51809685f4fbfa6703ffbf15ce714d816 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 27 May 2020 23:35:00 -0700 Subject: [PATCH 111/192] Xinfra Monitor Consumer Group Coordinator Assignment (#245) Xinfra Monitor Consumer Group Coordinator Assignment. 1. Implement Consumer Group Coordinator Assignment. 2. Bump up dependencies on Kafka and Kafka Client Signed-off-by: Andrew Choi li_andchoi@microsoft.com --- build.gradle | 4 +- .../common/ConsumerGroupCoordinatorUtils.java | 85 +++++++++++++++ .../linkedin/kmf/consumer/NewConsumer.java | 32 +++++- .../kmf/consumer/NewConsumerConfig.java | 41 +++++++ .../linkedin/kmf/services/ConsumeService.java | 19 +++- .../kmf/services/ConsumerFactoryImpl.java | 9 +- .../MultiClusterTopicManagementService.java | 57 ++++++++-- .../kmf/consumer/NewConsumerTest.java | 103 ++++++++++++++++++ 8 files changed, 322 insertions(+), 28 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/common/ConsumerGroupCoordinatorUtils.java create mode 100644 src/main/java/com/linkedin/kmf/consumer/NewConsumerConfig.java create mode 100644 src/test/java/com/linkedin/kmf/consumer/NewConsumerTest.java diff --git a/build.gradle b/build.gradle index d9c6aebb..c4c8d60c 100644 --- a/build.gradle +++ b/build.gradle @@ -38,8 +38,8 @@ allprojects { compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' - compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.0' + compile group: 'org.apache.kafka', name: 'kafka_2.13', version: '2.5.0' + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.5.0' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' } diff --git a/src/main/java/com/linkedin/kmf/common/ConsumerGroupCoordinatorUtils.java b/src/main/java/com/linkedin/kmf/common/ConsumerGroupCoordinatorUtils.java new file mode 100644 index 00000000..b2edc01d --- /dev/null +++ b/src/main/java/com/linkedin/kmf/common/ConsumerGroupCoordinatorUtils.java @@ -0,0 +1,85 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.common; + +import com.linkedin.kmf.consumer.NewConsumer; +import java.util.Collections; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.common.internals.Topic; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ConsumerGroupCoordinatorUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(NewConsumer.class); + private static final String CONSUMER_GROUP_PREFIX_CANDIDATE = "__shadow_consumer_group-"; + + /** + * https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala#L189 + * The consumer group string's hash code is used for this modulo operation. + * @param groupId kafka consumer group ID + * @param consumerOffsetsTopicPartitions number of partitions in the __consumer_offsets topic. + * @return hashed integer which represents a number, the Kafka's Utils.abs() value of which is the broker + * ID of the group coordinator, or the leader of the offsets topic partition. + */ + public static int partitionFor(String groupId, int consumerOffsetsTopicPartitions) { + + LOGGER.debug("Hashed and modulo output: {}", groupId.hashCode()); + return Utils.abs(groupId.hashCode()) % consumerOffsetsTopicPartitions; + } + + /** + * Instead of making targetGroupId an instance variable and then assigning it some value which this then looks up + * it can just be a parameter to a method + * hash(group.id) % (number of __consumer_offsets topic partitions). + * The partition's leader is the group coordinator + * Choose B s.t hash(A) % (number of __consumer_offsets topic partitions) == hash(B) % (number of __consumer_offsets topic partitions) + * @param targetGroupId the identifier of the target consumer group + * @param adminClient an Admin Client object + */ + public static String findCollision(String targetGroupId, AdminClient adminClient) + throws ExecutionException, InterruptedException { + if (targetGroupId.equals("")) { + throw new IllegalArgumentException("The target consumer group identifier cannot be empty: " + targetGroupId); + } + + int numOffsetsTopicPartitions = adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)) + .values() + .get(Topic.GROUP_METADATA_TOPIC_NAME) + .get() + .partitions() + .size(); + + // Extract invariant from loop + int targetConsumerOffsetsPartition = partitionFor(targetGroupId, numOffsetsTopicPartitions); + + // This doesn't need to be an instance variable because we throw this out this value at the end of computation + int groupSuffix = 0; + + // Extract return value so it's not computed twice, this reduces the possibility of bugs + String newConsumerGroup; + + // Use while(true) otherwise halting condition is hard to read. + while (true) { + // TODO: could play fancy StringBuilder games here to make this generate less garbage + newConsumerGroup = CONSUMER_GROUP_PREFIX_CANDIDATE + groupSuffix++; + int newGroupNamePartition = ConsumerGroupCoordinatorUtils.partitionFor(newConsumerGroup, numOffsetsTopicPartitions); + if (newGroupNamePartition == targetConsumerOffsetsPartition) { + break; + } + } + + return newConsumerGroup; + } +} + diff --git a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java index fac8fcf0..e1ddf56c 100644 --- a/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java +++ b/src/main/java/com/linkedin/kmf/consumer/NewConsumer.java @@ -10,11 +10,15 @@ package com.linkedin.kmf.consumer; +import com.linkedin.kmf.common.ConsumerGroupCoordinatorUtils; import java.time.Duration; import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -23,25 +27,42 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/* - * Wrap around the new consumer from Apache Kafka and implement the #KMBaseConsumer interface + +/** + * Wraps around the new consumer from Apache Kafka and implements the #KMBaseConsumer interface */ public class NewConsumer implements KMBaseConsumer { private final KafkaConsumer _consumer; private Iterator> _recordIter; - private static final Logger LOG = LoggerFactory.getLogger(NewConsumer.class); + private static final Logger LOGGER = LoggerFactory.getLogger(NewConsumer.class); private static long lastCommitted; - public NewConsumer(String topic, Properties consumerProperties) { + public NewConsumer(String topic, Properties consumerProperties, AdminClient adminClient) + throws ExecutionException, InterruptedException { + LOGGER.info("{} is being instantiated in the constructor..", this.getClass().getSimpleName()); + + NewConsumerConfig newConsumerConfig = new NewConsumerConfig(consumerProperties); + String targetConsumerGroupId = newConsumerConfig.getString(NewConsumerConfig.TARGET_CONSUMER_GROUP_ID_CONFIG); + + if (targetConsumerGroupId != null) { + consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, configureGroupId(targetConsumerGroupId, adminClient)); + } _consumer = new KafkaConsumer<>(consumerProperties); _consumer.subscribe(Collections.singletonList(topic)); } + static String configureGroupId(String targetConsumerGroupId, AdminClient adminClient) + throws ExecutionException, InterruptedException { + + return ConsumerGroupCoordinatorUtils.findCollision(targetConsumerGroupId, adminClient); + } + @Override public BaseConsumerRecord receive() { - if (_recordIter == null || !_recordIter.hasNext()) + if (_recordIter == null || !_recordIter.hasNext()) { _recordIter = _consumer.poll(Duration.ofMillis(Long.MAX_VALUE)).iterator(); + } ConsumerRecord record = _recordIter.next(); return new BaseConsumerRecord(record.topic(), record.partition(), record.offset(), record.key(), record.value()); @@ -81,5 +102,4 @@ public long lastCommitted() { public void updateLastCommit() { lastCommitted = System.currentTimeMillis(); } - } diff --git a/src/main/java/com/linkedin/kmf/consumer/NewConsumerConfig.java b/src/main/java/com/linkedin/kmf/consumer/NewConsumerConfig.java new file mode 100644 index 00000000..269a5068 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/consumer/NewConsumerConfig.java @@ -0,0 +1,41 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ +package com.linkedin.kmf.consumer; + +import java.util.Map; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; + + +/** + * Configuration for Xinfra Monitor New Consumer + */ +public class NewConsumerConfig extends AbstractConfig { + + private static final ConfigDef CONFIG_DEF; + + public static final String TARGET_CONSUMER_GROUP_ID_CONFIG = "target.consumer.group.id"; + public static final String TARGET_CONSUMER_GROUP_ID_CONFIG_DOC = + "When defined a consumer group is chosen such that it maps to the same group coordinator as the specified " + + "group coordinator."; + + static { + CONFIG_DEF = new ConfigDef().define(TARGET_CONSUMER_GROUP_ID_CONFIG, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + TARGET_CONSUMER_GROUP_ID_CONFIG_DOC); + } + + public NewConsumerConfig(Map props) { + super(CONFIG_DEF, props); + } +} + diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index fed2c6d0..1a6535d8 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -57,7 +57,7 @@ public class ConsumeService implements Service { private CommitAvailabilityMetrics _commitAvailabilityMetrics; private CommitLatencyMetrics _commitLatencyMetrics; private String _topic; - private String _name; + private final String _name; private static final String METRIC_GROUP_NAME = "consume-service"; private static Map tags; @@ -73,8 +73,8 @@ public class ConsumeService implements Service { * @param name Name of the Monitor instance * @param topicPartitionResult The completable future for topic partition * @param consumerFactory Consumer Factory object. - * @throws ExecutionException - * @throws InterruptedException + * @throws ExecutionException when attempting to retrieve the result of a task that aborted by throwing an exception + * @throws InterruptedException when a thread is waiting, sleeping, or otherwise occupied and the thread is interrupted */ public ConsumeService(String name, CompletableFuture topicPartitionResult, @@ -86,6 +86,8 @@ public ConsumeService(String name, _adminClient = consumerFactory.adminClient(); _running = new AtomicBoolean(false); + // Returns a new CompletionStage (topicPartitionFuture) which + // executes the given action - code inside run() - when this stage (topicPartitionResult) completes normally,. CompletableFuture topicPartitionFuture = topicPartitionResult.thenRun(() -> { MetricConfig metricConfig = new MetricConfig().samples(60).timeWindow(1000, TimeUnit.MILLISECONDS); List reporters = new ArrayList<>(); @@ -94,8 +96,10 @@ public ConsumeService(String name, tags = new HashMap<>(); tags.put(TAGS_NAME, name); _topic = consumerFactory.topic(); - _sensors = new ConsumeMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); - _commitLatencyMetrics = new CommitLatencyMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), consumerFactory.latencyPercentileGranularityMs()); + _sensors = new ConsumeMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), + consumerFactory.latencyPercentileGranularityMs()); + _commitLatencyMetrics = new CommitLatencyMetrics(metrics, tags, consumerFactory.latencyPercentileMaxMs(), + consumerFactory.latencyPercentileGranularityMs()); _commitAvailabilityMetrics = new CommitAvailabilityMetrics(metrics, tags); _consumeThread = new Thread(() -> { try { @@ -107,6 +111,7 @@ public ConsumeService(String name, _consumeThread.setDaemon(true); }); + // In a blocking fashion, waits for this topicPartitionFuture to complete, and then returns its result. topicPartitionFuture.get(); } @@ -124,6 +129,7 @@ record = _baseConsumer.receive(); _sensors._consumeError.record(); LOG.warn(_name + "/ConsumeService failed to receive record", e); /* Avoid busy while loop */ + //noinspection BusyWait Thread.sleep(CONSUME_THREAD_SLEEP_MS); continue; } @@ -134,7 +140,7 @@ record = _baseConsumer.receive(); try { avroRecord = Utils.genericRecordFromJson(record.value()); } catch (Exception exception) { - LOG.error("exception occurred while getting avro record.", exception); + LOG.error("An exception occurred while getting avro record.", exception); } if (avroRecord == null) { @@ -232,6 +238,7 @@ public synchronized void start() { } catch (InterruptedException | ExecutionException e) { LOG.error("Exception occurred while getting the topicDescriptionKafkaFuture for topic: {}", _topic, e); } + @SuppressWarnings("ConstantConditions") double partitionCount = topicDescription.partitions().size(); topicPartitionCount.add( new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), diff --git a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java index 49626551..f08e2472 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java @@ -28,16 +28,17 @@ public class ConsumerFactoryImpl implements ConsumerFactory { private final KMBaseConsumer _baseConsumer; - private String _topic; + private final String _topic; private static final String FALSE = "false"; private final int _latencyPercentileMaxMs; private final int _latencyPercentileGranularityMs; private static final String[] NON_OVERRIDABLE_PROPERTIES = new String[] {ConsumeServiceConfig.BOOTSTRAP_SERVERS_CONFIG, ConsumeServiceConfig.ZOOKEEPER_CONNECT_CONFIG}; - private int _latencySlaMs; + private final int _latencySlaMs; private static AdminClient adminClient; private static final Logger LOG = LoggerFactory.getLogger(ConsumerFactoryImpl.class); + @SuppressWarnings("rawtypes") public ConsumerFactoryImpl(Map props) throws Exception { LOG.info("Creating AdminClient."); adminClient = AdminClient.create(props); @@ -80,7 +81,9 @@ public ConsumerFactoryImpl(Map props) throws Exception { props.forEach(consumerProps::putIfAbsent); } - _baseConsumer = (KMBaseConsumer) Class.forName(consumerClassName).getConstructor(String.class, Properties.class).newInstance(_topic, consumerProps); + _baseConsumer = (KMBaseConsumer) Class.forName(consumerClassName) + .getConstructor(String.class, Properties.class, AdminClient.class) + .newInstance(_topic, consumerProps, adminClient()); } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index bc6f923c..088e5f98 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import kafka.admin.AdminUtils; import kafka.admin.BrokerMetadata; +import kafka.controller.ReplicaAssignment; import kafka.server.ConfigType; import kafka.zk.KafkaZkClient; import org.apache.kafka.clients.admin.AdminClient; @@ -342,8 +343,8 @@ private Set getAvailableBrokers() throws ExecutionException, InterruptedEx } void maybeReassignPartitionAndElectLeader() throws Exception { - try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, null)) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); @@ -416,8 +417,8 @@ void maybeElectLeader() throws Exception { return; } - try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, null)) { if (!zkClient.reassignPartitionsInProgress()) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); @@ -462,9 +463,10 @@ private static void reassignPartitions(KafkaZkClient zkClient, Collection } scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); - scala.collection.Map> currentAssignment = zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); - String currentAssignmentJson = formatAsReassignmentJson(topic, currentAssignment); - String newAssignmentJson = formatAsReassignmentJson(topic, assignedReplicas); + scala.collection.Map + currentAssignment = zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); + String currentAssignmentJson = formatAsOldAssignmentJson(topic, currentAssignment); + String newAssignmentJson = formatAsNewReassignmentJson(topic, assignedReplicas); LOGGER.info("Reassign partitions for topic " + topic); LOGGER.info("Current partition replica assignment " + currentAssignmentJson); @@ -521,14 +523,14 @@ static boolean someBrokerNotElectedLeader(List partitionInfo * {"topic":"kmf-topic","partition":0,"replicas":[2,0]}]} * */ - private static String formatAsReassignmentJson(String topic, scala.collection.Map> partitionsToBeReassigned) { + private static String formatAsOldAssignmentJson(String topic, scala.collection.Map partitionsToBeReassigned) { StringBuilder bldr = new StringBuilder(); bldr.append("{\"version\":1,\"partitions\":[\n"); for (int partition = 0; partition < partitionsToBeReassigned.size(); partition++) { bldr.append(" {\"topic\":\"").append(topic).append("\",\"partition\":").append(partition).append(",\"replicas\":["); - scala.collection.Seq replicas = partitionsToBeReassigned.apply(partition); - for (int replicaIndex = 0; replicaIndex < replicas.size(); replicaIndex++) { - Object replica = replicas.apply(replicaIndex); + ReplicaAssignment replicas = partitionsToBeReassigned.apply(partition); + for (int replicaIndex = 0; replicaIndex < replicas.replicas().size(); replicaIndex++) { + Object replica = replicas.replicas().apply(replicaIndex); bldr.append(replica).append(","); } bldr.setLength(bldr.length() - 1); @@ -538,5 +540,38 @@ private static String formatAsReassignmentJson(String topic, scala.collection.Ma bldr.append("]}"); return bldr.toString(); } + + /** + * @param topic Kafka topic + * @param partitionsToReassign a map from partition (int) to new replica list (int seq) + * + * @return a json string with the same format as output of kafka.utils.ZkUtils.formatAsReassignmentJson + * + * Example: + *
+     *   {"version":1,"partitions":[
+     *     {"topic":"kmf-topic","partition":1,"replicas":[0,1]},
+     *     {"topic":"kmf-topic","partition":2,"replicas":[1,2]},
+     *     {"topic":"kmf-topic","partition":0,"replicas":[2,0]}]}
+     * 
+ */ + private static String formatAsNewReassignmentJson(String topic, scala.collection.Map> partitionsToReassign) { + StringBuilder builder = new StringBuilder(); + builder.append("{\"version\":1,\"partitions\":[\n"); + for (int partition = 0; partition < partitionsToReassign.size(); partition++) { + builder.append(" {\"topic\":\"").append(topic).append("\",\"partition\":").append(partition).append(",\"replicas\":["); + Seq replicas = partitionsToReassign.apply(partition); + for (int replicaIndex = 0; replicaIndex < replicas.size(); replicaIndex++) { + Object replica = replicas.apply(replicaIndex); + builder.append(replica).append(","); + } + builder.setLength(builder.length() - 1); + builder.append("]},\n"); + } + builder.setLength(builder.length() - 2); + builder.append("]}"); + return builder.toString(); + } + } } diff --git a/src/test/java/com/linkedin/kmf/consumer/NewConsumerTest.java b/src/test/java/com/linkedin/kmf/consumer/NewConsumerTest.java new file mode 100644 index 00000000..b7e8b4cc --- /dev/null +++ b/src/test/java/com/linkedin/kmf/consumer/NewConsumerTest.java @@ -0,0 +1,103 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.consumer; + +import com.linkedin.kmf.common.ConsumerGroupCoordinatorUtils; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.internals.Topic; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +@Test +public class NewConsumerTest { + private static final int NUM_OFFSETS_TOPIC_PARTITIONS = 5; + private static final String TARGET_CONSUMER_GROUP_ID = "target-group-id"; + + @BeforeMethod + public void beforeMethod() { + System.out.println("Running beforeMethod of " + this.getClass()); + } + + @AfterMethod + public void afterMethod() { + System.out.println("Finished running testConsumerGroupCoordinatorHashing() of " + this.getClass()); + } + + @SuppressWarnings("unchecked") + @Test + public void testConsumerGroupCoordinatorHashing() throws ExecutionException, InterruptedException { + Properties consumerProperties = new Properties(); + + AdminClient adminClient = Mockito.mock(AdminClient.class); + + /* + * Mock the behavior of AdminClient only. + */ + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME))) + .thenReturn(Mockito.mock(DescribeTopicsResult.class)); + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)).values()) + .thenReturn(Mockito.mock(Map.class)); + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)) + .values() + .get(Topic.GROUP_METADATA_TOPIC_NAME)).thenReturn(Mockito.mock(KafkaFutureImpl.class)); + + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)) + .values() + .get(Topic.GROUP_METADATA_TOPIC_NAME) + .get()).thenReturn(Mockito.mock(TopicDescription.class)); + + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)) + .values() + .get(Topic.GROUP_METADATA_TOPIC_NAME) + .get() + .partitions()).thenReturn(Mockito.mock(List.class)); + + Mockito.when(adminClient.describeTopics(Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)) + .values() + .get(Topic.GROUP_METADATA_TOPIC_NAME) + .get() + .partitions() + .size()).thenReturn(NUM_OFFSETS_TOPIC_PARTITIONS); + + consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, + NewConsumer.configureGroupId(TARGET_CONSUMER_GROUP_ID, adminClient)); + System.out.println("Consumer properties after configuration: " + consumerProperties); + Assert.assertNotNull(consumerProperties.get(ConsumerConfig.GROUP_ID_CONFIG)); + + // Testing I: run partitionsFor() on the result to make sure they are the same + int hashedResult = + ConsumerGroupCoordinatorUtils.partitionFor(consumerProperties.get(ConsumerConfig.GROUP_ID_CONFIG).toString(), + NUM_OFFSETS_TOPIC_PARTITIONS); + int hashedResult2 = + ConsumerGroupCoordinatorUtils.partitionFor(TARGET_CONSUMER_GROUP_ID, NUM_OFFSETS_TOPIC_PARTITIONS); + + Assert.assertEquals(hashedResult, hashedResult2); + System.out.println("Modulo result as an absolute value: " + hashedResult); + System.out.println("Modulo result as an absolute value: " + hashedResult2); + + // Testing II: Also test that the groupIds are different. + Assert.assertNotEquals(TARGET_CONSUMER_GROUP_ID, consumerProperties.get(ConsumerConfig.GROUP_ID_CONFIG)); + + } +} From d037baf9150bec502db36bfb16eb54e8b5b94870 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 28 May 2020 22:35:45 -0700 Subject: [PATCH 112/192] Partition Assignments when adding new topic partitions (#249) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Random assignment of partition assignments when adding partitions periodically Justifications: regardless of the partition/replica assignments here, maybeReassignPartitionAndElectLeader() will reassign the partition as needed periodically. 🚣🏻 Signed-off-by: Andrew Choi --- .../MultiClusterTopicManagementService.java | 54 +++++++++++++++++-- ...ultiClusterTopicManagementServiceTest.java | 30 ++++++++++- 2 files changed, 77 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 088e5f98..ce7ee6f2 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -37,6 +37,7 @@ import kafka.zk.KafkaZkClient; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.CreatePartitionsResult; import org.apache.kafka.clients.admin.ElectLeadersOptions; import org.apache.kafka.clients.admin.ElectLeadersResult; import org.apache.kafka.clients.admin.NewPartitions; @@ -302,7 +303,6 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup LOGGER.info("{} will increase partition of the topic {} in the cluster from {}" + " to {}.", this.getClass().toString(), _topic, partitionNum, minPartitionNum); Set blackListedBrokers = _topicFactory.getBlackListedBrokers(_zkConnect); - List> replicaAssignment = new ArrayList<>(new ArrayList<>()); Set brokers = new HashSet<>(); for (Node broker : _adminClient.describeCluster().nodes().get()) { BrokerMetadata brokerMetadata = new BrokerMetadata( @@ -314,11 +314,57 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup if (!blackListedBrokers.isEmpty()) { brokers.removeIf(broker -> blackListedBrokers.contains(broker.id())); } + + List> newPartitionAssignments = newPartitionAssignments(minPartitionNum, partitionNum, brokers, _replicationFactor); + + NewPartitions newPartitions = NewPartitions.increaseTo(minPartitionNum, newPartitionAssignments); + Map newPartitionsMap = new HashMap<>(); - NewPartitions newPartitions = NewPartitions.increaseTo(minPartitionNum, replicaAssignment); newPartitionsMap.put(_topic, newPartitions); - _adminClient.createPartitions(newPartitionsMap); + CreatePartitionsResult createPartitionsResult = _adminClient.createPartitions(newPartitionsMap); + + createPartitionsResult.all().get(); + } + } + + static List> newPartitionAssignments(int minPartitionNum, int partitionNum, + Set brokers, int rf) { + + // The replica assignments for the new partitions, and not the old partitions. + // .increaseTo(6, asList(asList(1, 2), + // asList(2, 3), + // asList(3, 1))) + // partition 3's preferred leader will be broker 1, + // partition 4's preferred leader will be broker 2 and + // partition 5's preferred leader will be broker 3. + List> newPartitionAssignments = new ArrayList<>(new ArrayList<>()); + int partitionDifference = minPartitionNum - partitionNum; + + // leader assignments - + for (BrokerMetadata brokerMetadata : brokers) { + List replicas = new ArrayList<>(); + // leader replica/broker - + replicas.add(brokerMetadata.id()); + newPartitionAssignments.add(replicas); + if (newPartitionAssignments.size() == partitionDifference) { + break; + } } + + // follower assignments - + // Regardless of the partition/replica assignments here, maybeReassignPartitionAndElectLeader() + // will reassign the partition as needed periodically. + for (List replicas : newPartitionAssignments) { + for (BrokerMetadata broker : brokers) { + if (!replicas.contains(broker.id())) { + replicas.add(broker.id()); + } + if (replicas.size() == rf) { + break; + } + } + } + return newPartitionAssignments; } /** @@ -329,8 +375,6 @@ void maybeAddPartitions(int minPartitionNum) throws ExecutionException, Interrup */ int numPartitions() throws InterruptedException, ExecutionException { - // TODO (andrewchoi5): connect this to unit testing method for testing maybeAddPartitions! - return _adminClient.describeTopics(Collections.singleton(_topic)).values().get(_topic).get().partitions().size(); } diff --git a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java index da172d06..e97bb407 100644 --- a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java @@ -12,9 +12,11 @@ import com.linkedin.kmf.topicfactory.TopicFactory; import java.util.Collections; -import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import java.util.Set; +import kafka.admin.BrokerMetadata; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.DescribeClusterResult; @@ -29,6 +31,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import scala.Option; /** @@ -37,6 +40,7 @@ @SuppressWarnings("unchecked") @Test public class MultiClusterTopicManagementServiceTest { + private static final String SERVICE_TEST_TOPIC = "xinfra-monitor-Multi-Cluster-Topic-Management-Service-Test-topic"; private static Set nodeSet; private MultiClusterTopicManagementService.TopicManagementHelper _topicManagementHelper; @@ -50,12 +54,15 @@ private void startTest() { _kafkaFutureMap = Mockito.mock(Map.class); _kafkaFuture = Mockito.mock(KafkaFuture.class); - nodeSet = new HashSet<>(); + nodeSet = new LinkedHashSet<>(); nodeSet.add(new Node(1, "host-1", 2132)); nodeSet.add(new Node(2, "host-2", 2133)); nodeSet.add(new Node(3, "host-3", 2134)); nodeSet.add(new Node(4, "host-4", 2135)); nodeSet.add(new Node(5, "host-5", 2136)); + nodeSet.add(new Node(6, "host-5", 2136)); + nodeSet.add(new Node(7, "host-5", 2136)); + nodeSet.add(new Node(8, "host-5", 2136)); _topicManagementHelper = Mockito.mock(MultiClusterTopicManagementService.TopicManagementHelper.class); _topicManagementHelper._topic = SERVICE_TEST_TOPIC; @@ -69,6 +76,25 @@ private void finishTest() { System.out.println("Finished " + this.getClass().getCanonicalName().toLowerCase() + "."); } + @Test(invocationCount = 2) + protected void maybeAddPartitionsTest() { + Set brokerMetadataSet = new LinkedHashSet<>(); + for (Node broker : nodeSet) { + brokerMetadataSet.add(new BrokerMetadata(broker.id(), Option.apply(broker.rack()))); + } + List> newPartitionAssignments = + MultiClusterTopicManagementService.TopicManagementHelper.newPartitionAssignments(11, 5, brokerMetadataSet, 4); + Assert.assertNotNull(newPartitionAssignments); + + System.out.println(newPartitionAssignments); + Assert.assertEquals(newPartitionAssignments.get(0).get(0).intValue(), 1); + Assert.assertEquals(newPartitionAssignments.get(1).get(0).intValue(), 2); + Assert.assertEquals(newPartitionAssignments.get(2).get(0).intValue(), 3); + Assert.assertEquals(newPartitionAssignments.get(3).get(0).intValue(), 4); + Assert.assertEquals(newPartitionAssignments.get(4).get(0).intValue(), 5); + Assert.assertEquals(newPartitionAssignments.get(5).get(0).intValue(), 6); + } + @Test protected void MultiClusterTopicManagementServiceTopicCreationTest() throws Exception { From 0877cde68d89a3fd9f64f703f47ba1394b4c8a9d Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 29 May 2020 00:49:42 -0700 Subject: [PATCH 113/192] optional Signed-off-by: Andrew Choi --- src/main/java/com/linkedin/kmf/common/Utils.java | 5 +++-- .../kmf/services/MultiClusterTopicManagementService.java | 6 ++++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 0d2c2fd5..8aa0287f 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -88,6 +88,7 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, throws ExecutionException, InterruptedException { try { if (adminClient.listTopics().names().get().contains(topic)) { + LOG.info("AdminClient indicates that {} already exists in the cluster. Topic config: {}", topic, topicConfig); return getPartitionNumForTopic(adminClient, topic); } int brokerCount = Utils.getBrokerCount(adminClient); @@ -107,7 +108,7 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, } catch (TopicExistsException e) { /* There is a race condition with the consumer. */ - LOG.debug("Monitoring topic " + topic + " already exists in the cluster.", e); + LOG.info("Monitoring topic " + topic + " already exists in the cluster.", e); return getPartitionNumForTopic(adminClient, topic); } LOG.info("Created monitoring topic {} in cluster with {} partitions and replication factor of {}.", topic, @@ -115,7 +116,7 @@ public static int createTopicIfNotExists(String topic, short replicationFactor, return partitionCount; } finally { - LOG.info("Completed the topic creation if it doesn't exist for {}", topic); + LOG.info("Completed the topic creation if it doesn't exist for {}.", topic); } } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index ce7ee6f2..ee8f5d47 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -54,6 +54,7 @@ import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Option; import scala.Option$; import scala.collection.Seq; @@ -387,8 +388,9 @@ private Set getAvailableBrokers() throws ExecutionException, InterruptedEx } void maybeReassignPartitionAndElectLeader() throws Exception { - try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, null)) { + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), + com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, + Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, Option.apply(null))) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); From 8793d3234eb196f2668daa137d576510e16e7c5d Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 29 May 2020 16:54:51 -0700 Subject: [PATCH 114/192] apache kafka scala version 2.13 with code version 2.4.1 (#251) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit apache.kafka use version : '2.4.1' justification: the internal kmf multiproduct currently employs scala version _2.11. Moreover, Anasi (or ELR tool) doesn't allow for intaking scala _2.11 with apache kafka code version 2.5.0 because the maven service doesn't have, and thus does not release, the scala _2.11 version 2.5.0 . we cannot bump up to scala version _2.13 in the kafka-monitoring multiproduct because there are many, many multiproduct dependencies on the scala version _2.11 that have not been ELR.ed yet. Justifications for merging this: the PR won’t break any dependencies because the bump to 2.5.0 happened very recently in this repository 2 - internal MP (xinfra monitor) can be bumped up to scala kafka_2.12 but even that requires modifications to several kmf-depended multiproducts, including trackingrest, container, etc.. which could easily take a long time to all complete. bumping up kmf internal mp to scala 2.12 requires these changes which could take some time to code-complete, merge, and get released. What went wrong: Execution failed for task ':likafka-monitoring:kafka-monitoring-impl_2.12:compileJava'. Could not resolve all files for configuration ':likafka-monitoring:kafka-monitoring-impl_2.12:compileClasspath'. Could not find com.linkedin.linkedin-kafka-clients:linkedin-kafka-clients-message-processor_2.12:8.0.204. Required by: project :likafka-monitoring:kafka-monitoring-impl_2.12 > com.linkedin.kafka-rest:tracking-rest-impl:0.3.150 Could not find com.linkedin.container:tracker-consumer-internal_2.12:36.12.6. Required by: project :likafka-monitoring:kafka-monitoring-impl_2.12 > com.linkedin.kafka-rest:tracking-rest-impl:0.3.150 > com.linkedin.container:tracker-consumer-api:37.1.63 Could not find com.linkedin.likafka-cruise-control:likafka-cruise-control-impl_2.12:1.0.17. Required by: project :likafka-monitoring:kafka-monitoring-impl_2.12 > com.linkedin.kafka-rest:tracking-rest-impl:0.3.150 > com.linkedin.kafka-server:kafka-impl:2.0.0.71 lets have kmf mp stay in 2.11 for now. --- build.gradle | 4 ++-- .../kmf/services/MultiClusterTopicManagementService.java | 9 ++++----- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/build.gradle b/build.gradle index c4c8d60c..7a58826a 100644 --- a/build.gradle +++ b/build.gradle @@ -38,8 +38,8 @@ allprojects { compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' - compile group: 'org.apache.kafka', name: 'kafka_2.13', version: '2.5.0' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.5.0' + compile group: 'org.apache.kafka', name: 'kafka_2.13', version: '2.4.1' + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.1' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index ee8f5d47..9b7ad805 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -54,7 +54,6 @@ import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Option; import scala.Option$; import scala.collection.Seq; @@ -388,9 +387,9 @@ private Set getAvailableBrokers() throws ExecutionException, InterruptedEx } void maybeReassignPartitionAndElectLeader() throws Exception { - try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, - Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, Option.apply(null))) { + Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); @@ -463,8 +462,8 @@ void maybeElectLeader() throws Exception { return; } - try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSaslEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, - com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null, null)) { + try (KafkaZkClient zkClient = KafkaZkClient.apply(_zkConnect, JaasUtils.isZkSecurityEnabled(), com.linkedin.kmf.common.Utils.ZK_SESSION_TIMEOUT_MS, + com.linkedin.kmf.common.Utils.ZK_CONNECTION_TIMEOUT_MS, Integer.MAX_VALUE, Time.SYSTEM, METRIC_GROUP_NAME, "SessionExpireListener", null)) { if (!zkClient.reassignPartitionsInProgress()) { List partitionInfoList = _adminClient .describeTopics(Collections.singleton(_topic)).all().get().get(_topic).partitions(); From 31efd33d2a704523cd94bf588a92fefb7f6bb28c Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sat, 30 May 2020 19:52:55 -0700 Subject: [PATCH 115/192] scala 2.21 (#254) Signed-off-by: Andrew Choi --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 7a58826a..ec106864 100644 --- a/build.gradle +++ b/build.gradle @@ -38,7 +38,7 @@ allprojects { compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' - compile group: 'org.apache.kafka', name: 'kafka_2.13', version: '2.4.1' + compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.4.1' compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.1' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' From b4f9d9e8bede94f22ecfe5729f7f89f93cf0fae4 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 31 May 2020 01:17:29 -0700 Subject: [PATCH 116/192] Use of org.apache.kafka - version: 2.3.1 #255 Use of org.apache.kafka - version: 2.3.1 Justification: Current linkedin kafka repository, "kafka": "com.linkedin.kafka:kafka_2.12:2.3.0.20", which internal kmf mp depends on, still uses version 2.3.0.20. This linkedin kafka version uses def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, Seq[Int]]] as its parameters for the method getPartitionAssignmentForTopics. However, the compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.4.1' uses def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, ReplicaAssignment]] Thus, apache kafka 2.4.1 is not backward compatible with internal kmf MP and linkedin kafka inside kmf MP. -> there is a discrepancy in method definition of linkedin kafka versus apache kafka. Solution: With the older version of apache kafka 2.3.1, there is no discrepancy between the two products. I have checked that there is no breaking changes when reverting back to apache kafka version 2.3.1. I have added a todo item (andrewchoi5) to uncomment the related method when Xinfra Monitor is upgraded to 'org.apache.kafka' 'kafka_2.12' version '2.4.1' later, at which point we expect the linkedin kafka to have the parameters for getPartitionAssignmentForTopics updated. /** * Gets partition the assignments for the given topics. * @param topics the topics whose partitions we wish to get the assignments for. * @return the partition assignment for each partition from the given topics. */ def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, Seq[Int]]] = { val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic))) val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq) getDataResponses.flatMap { getDataResponse => val topic = getDataResponse.ctx.get.asInstanceOf[String] if (getDataResponse.resultCode == Code.OK) { val partitionMap = TopicZNode.decode(topic, getDataResponse.data).map { case (k, v) => (k.partition, v) } Map(topic -> partitionMap) } else if (getDataResponse.resultCode == Code.NONODE) { Map.empty[String, Map[Int, Seq[Int]]] } else { throw getDataResponse.resultException.get } }.toMap } Signed-off-by: Andrew Choi --- build.gradle | 2 +- .../MultiClusterTopicManagementService.java | 43 ++++++++++--------- 2 files changed, 23 insertions(+), 22 deletions(-) diff --git a/build.gradle b/build.gradle index ec106864..d1410b36 100644 --- a/build.gradle +++ b/build.gradle @@ -38,7 +38,7 @@ allprojects { compile 'net.savantly:graphite-client:1.1.0-RELEASE' compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' - compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.4.1' + compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.1' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 9b7ad805..81349ec3 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -32,7 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import kafka.admin.AdminUtils; import kafka.admin.BrokerMetadata; -import kafka.controller.ReplicaAssignment; import kafka.server.ConfigType; import kafka.zk.KafkaZkClient; import org.apache.kafka.clients.admin.AdminClient; @@ -508,9 +507,9 @@ private static void reassignPartitions(KafkaZkClient zkClient, Collection } scala.collection.immutable.Set topicList = new scala.collection.immutable.Set.Set1<>(topic); - scala.collection.Map - currentAssignment = zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); - String currentAssignmentJson = formatAsOldAssignmentJson(topic, currentAssignment); + scala.collection.Map> currentAssignment = + zkClient.getPartitionAssignmentForTopics(topicList).apply(topic); + String currentAssignmentJson = formatAsNewReassignmentJson(topic, currentAssignment); String newAssignmentJson = formatAsNewReassignmentJson(topic, assignedReplicas); LOGGER.info("Reassign partitions for topic " + topic); @@ -568,23 +567,25 @@ static boolean someBrokerNotElectedLeader(List partitionInfo * {"topic":"kmf-topic","partition":0,"replicas":[2,0]}]} * */ - private static String formatAsOldAssignmentJson(String topic, scala.collection.Map partitionsToBeReassigned) { - StringBuilder bldr = new StringBuilder(); - bldr.append("{\"version\":1,\"partitions\":[\n"); - for (int partition = 0; partition < partitionsToBeReassigned.size(); partition++) { - bldr.append(" {\"topic\":\"").append(topic).append("\",\"partition\":").append(partition).append(",\"replicas\":["); - ReplicaAssignment replicas = partitionsToBeReassigned.apply(partition); - for (int replicaIndex = 0; replicaIndex < replicas.replicas().size(); replicaIndex++) { - Object replica = replicas.replicas().apply(replicaIndex); - bldr.append(replica).append(","); - } - bldr.setLength(bldr.length() - 1); - bldr.append("]},\n"); - } - bldr.setLength(bldr.length() - 2); - bldr.append("]}"); - return bldr.toString(); - } + + // TODO (andrewchoi5): uncomment this method when Xinfra Monitor is upgraded to 'org.apache.kafka' 'kafka_2.12' version '2.4.1' +// private static String formatAsOldAssignmentJson(String topic, scala.collection.Map partitionsToBeReassigned) { +// StringBuilder bldr = new StringBuilder(); +// bldr.append("{\"version\":1,\"partitions\":[\n"); +// for (int partition = 0; partition < partitionsToBeReassigned.size(); partition++) { +// bldr.append(" {\"topic\":\"").append(topic).append("\",\"partition\":").append(partition).append(",\"replicas\":["); +// ReplicaAssignment replicas = partitionsToBeReassigned.apply(partition); +// for (int replicaIndex = 0; replicaIndex < replicas.replicas().size(); replicaIndex++) { +// Object replica = replicas.replicas().apply(replicaIndex); +// bldr.append(replica).append(","); +// } +// bldr.setLength(bldr.length() - 1); +// bldr.append("]},\n"); +// } +// bldr.setLength(bldr.length() - 2); +// bldr.append("]}"); +// return bldr.toString(); +// } /** * @param topic Kafka topic From 80a6ba0ff6eab4706a3c2deb4ecbf076d1d1c543 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Sun, 31 May 2020 11:48:26 -0700 Subject: [PATCH 117/192] Update Apache kafka kafka-clients version 2.3.1 #256 Our internal kmf mp replies not on apache kafka, but linkedin/kafka which still to this date replies on the old copy - 2.3.*. It makes more sense to use java kafka-clients 2.3.1 rather than 2.4.1 to better monitor our kafka clusters. compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.3.1' Signed-off-by: Andrew Choi --- build.gradle | 2 +- .../com/linkedin/kmf/services/ConsumeService.java | 5 ++--- .../services/MultiClusterTopicManagementService.java | 12 +++--------- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/build.gradle b/build.gradle index d1410b36..091bbf0f 100644 --- a/build.gradle +++ b/build.gradle @@ -39,7 +39,7 @@ allprojects { compile 'com.timgroup:java-statsd-client:3.0.1' compile 'com.signalfx.public:signalfx-codahale:0.0.47' compile group: 'org.apache.kafka', name: 'kafka_2.12', version: '2.3.1' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.4.1' + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.3.1' testCompile 'org.mockito:mockito-core:2.24.0' testCompile 'org.testng:testng:6.8.8' } diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeService.java b/src/main/java/com/linkedin/kmf/services/ConsumeService.java index 1a6535d8..307dffab 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumeService.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumeService.java @@ -37,7 +37,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Total; import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -241,8 +241,7 @@ public synchronized void start() { @SuppressWarnings("ConstantConditions") double partitionCount = topicDescription.partitions().size(); topicPartitionCount.add( - new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), - new CumulativeSum(partitionCount)); + new MetricName("topic-partitions-count", METRIC_GROUP_NAME, "The total number of partitions for the topic.", tags), new Total(partitionCount)); } } diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 81349ec3..d2eb667d 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -37,12 +37,10 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.CreatePartitionsResult; -import org.apache.kafka.clients.admin.ElectLeadersOptions; -import org.apache.kafka.clients.admin.ElectLeadersResult; +import org.apache.kafka.clients.admin.ElectPreferredLeadersResult; import org.apache.kafka.clients.admin.NewPartitions; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; @@ -481,13 +479,9 @@ private void triggerPreferredLeaderElection(List partitionIn for (TopicPartitionInfo javaPartitionInfo : partitionInfoList) { partitions.add(new TopicPartition(partitionTopic, javaPartitionInfo.partition())); } + ElectPreferredLeadersResult electPreferredLeadersResult = _adminClient.electPreferredLeaders(partitions); - ElectLeadersOptions newOptions = new ElectLeadersOptions(); - ElectionType electionType = ElectionType.PREFERRED; - Set topicPartitions = new HashSet<>(partitions); - ElectLeadersResult electLeadersResult = _adminClient.electLeaders(electionType, topicPartitions, newOptions); - - LOGGER.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electLeadersResult.all().get()); + LOGGER.info("{}: triggerPreferredLeaderElection - {}", this.getClass().toString(), electPreferredLeadersResult.all().get()); } private static void reassignPartitions(KafkaZkClient zkClient, Collection brokers, String topic, int partitionCount, int replicationFactor) { From ac73d65c271df8ba2bc6847776e156ad7a595c93 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 1 Jun 2020 12:37:21 -0700 Subject: [PATCH 118/192] Xinfra Monitor Rebranding Operation from Kafka Monitor Signed-off-by: Andrew Choi Xinfra Monitor Rebranding Operation from Kafka Monitor Signed-off-by: Andrew Choi --- bin/kafka-monitor-start.sh | 2 +- bin/windows/kafka-monitor-start.bat | 2 +- bin/windows/kmf-run-class.bat | 2 +- .../{KafkaMonitor.java => XinfraMonitor.java} | 14 +++++----- .../com/linkedin/kmf/KafkaMonitorTest.java | 28 +++++++++---------- 5 files changed, 24 insertions(+), 24 deletions(-) rename src/main/java/com/linkedin/kmf/{KafkaMonitor.java => XinfraMonitor.java} (94%) diff --git a/bin/kafka-monitor-start.sh b/bin/kafka-monitor-start.sh index eb4863dc..0c389bfa 100755 --- a/bin/kafka-monitor-start.sh +++ b/bin/kafka-monitor-start.sh @@ -9,4 +9,4 @@ base_dir=$(dirname $0) -exec $base_dir/kmf-run-class.sh com/linkedin/kmf/KafkaMonitor $@ +exec $base_dir/kmf-run-class.sh com/linkedin/kmf/XinfraMonitor $@ diff --git a/bin/windows/kafka-monitor-start.bat b/bin/windows/kafka-monitor-start.bat index abba5640..d8928ee2 100644 --- a/bin/windows/kafka-monitor-start.bat +++ b/bin/windows/kafka-monitor-start.bat @@ -19,7 +19,7 @@ IF [%1] EQU [] ( EXIT /B 1 ) -set COMMAND=%BASE_DIR%\kmf-run-class.bat com.linkedin.kmf.KafkaMonitor %* +set COMMAND=%BASE_DIR%\kmf-run-class.bat com.linkedin.kmf.XinfraMonitor %* rem echo basedir: %BASE_DIR% diff --git a/bin/windows/kmf-run-class.bat b/bin/windows/kmf-run-class.bat index 65fb1cf9..619f70d3 100644 --- a/bin/windows/kmf-run-class.bat +++ b/bin/windows/kmf-run-class.bat @@ -10,7 +10,7 @@ REM an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either expre setlocal enabledelayedexpansion IF [%1] EQU [] ( - echo USAGE: %0 com.linkedin.kmf.KafkaMonitor config/kafka-monitor.properties + echo USAGE: %0 com.linkedin.kmf.XinfraMonitor config/kafka-monitor.properties EXIT /B 1 ) diff --git a/src/main/java/com/linkedin/kmf/KafkaMonitor.java b/src/main/java/com/linkedin/kmf/XinfraMonitor.java similarity index 94% rename from src/main/java/com/linkedin/kmf/KafkaMonitor.java rename to src/main/java/com/linkedin/kmf/XinfraMonitor.java index db7c745d..626bf105 100644 --- a/src/main/java/com/linkedin/kmf/KafkaMonitor.java +++ b/src/main/java/com/linkedin/kmf/XinfraMonitor.java @@ -39,8 +39,8 @@ * This is the main entry point of the monitor. It reads the configuration and manages the life cycle of the monitoring * applications. */ -public class KafkaMonitor { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMonitor.class); +public class XinfraMonitor { + private static final Logger LOG = LoggerFactory.getLogger(XinfraMonitor.class); public static final String CLASS_NAME_CONFIG = "class.name"; private static final String METRIC_GROUP_NAME = "kafka-monitor"; private static final String JMX_PREFIX = "kmf"; @@ -63,7 +63,7 @@ public class KafkaMonitor { * @throws Exception */ @SuppressWarnings({"rawtypes", "unchecked"}) - public KafkaMonitor(Map allClusterProps) throws Exception { + public XinfraMonitor(Map allClusterProps) throws Exception { _apps = new ConcurrentHashMap<>(); _services = new ConcurrentHashMap<>(); @@ -179,7 +179,7 @@ public void awaitShutdown() { @SuppressWarnings("rawtypes") public static void main(String[] args) throws Exception { if (args.length <= 0) { - LOG.info("USAGE: java [options] " + KafkaMonitor.class.getName() + " config/kafka-monitor.properties"); + LOG.info("USAGE: java [options] " + XinfraMonitor.class.getName() + " config/kafka-monitor.properties"); return; } @@ -194,11 +194,11 @@ public static void main(String[] args) throws Exception { @SuppressWarnings("unchecked") Map props = new ObjectMapper().readValue(buffer.toString(), Map.class); - KafkaMonitor kafkaMonitor = new KafkaMonitor(props); - kafkaMonitor.start(); + XinfraMonitor xinfraMonitor = new XinfraMonitor(props); + xinfraMonitor.start(); LOG.info("Xinfra Monitor (KafkaMonitor) started."); - kafkaMonitor.awaitShutdown(); + xinfraMonitor.awaitShutdown(); } } diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index e8d16934..fa6e65c0 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -24,41 +24,41 @@ public class KafkaMonitorTest { @Test public void lifecycleTest() throws Exception { - KafkaMonitor kafkaMonitor = kafkaMonitor(); + XinfraMonitor xinfraMonitor = kafkaMonitor(); /* Nothing should be started */ org.testng.Assert.assertEquals(FakeService.startCount.get(), 0); org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); /* Should accept but ignore start because start has not been called */ - kafkaMonitor.stop(); + xinfraMonitor.stop(); org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); /* Should start */ - kafkaMonitor.start(); + xinfraMonitor.start(); org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); /* Should allow start to be called more than once */ - kafkaMonitor.stop(); - kafkaMonitor.stop(); + xinfraMonitor.stop(); + xinfraMonitor.stop(); org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); org.testng.Assert.assertEquals(FakeService.stopCount.get(), 1); /* Should be allowed to shutdown more than once. */ - kafkaMonitor.awaitShutdown(); - kafkaMonitor.awaitShutdown(); + xinfraMonitor.awaitShutdown(); + xinfraMonitor.awaitShutdown(); } @Test public void awaitShutdownOtherThread() throws Exception { - final KafkaMonitor kafkaMonitor = kafkaMonitor(); + final XinfraMonitor xinfraMonitor = kafkaMonitor(); final AtomicReference error = new AtomicReference<>(); Thread t = new Thread("test awaitshutdown thread") { @Override public void run() { try { - kafkaMonitor.awaitShutdown(); + xinfraMonitor.awaitShutdown(); } catch (Throwable t) { error.set(t); } @@ -66,21 +66,21 @@ public void run() { }; t.start(); - kafkaMonitor.start(); + xinfraMonitor.start(); Thread.sleep(100); - kafkaMonitor.stop(); + xinfraMonitor.stop(); t.join(500); org.testng.Assert.assertFalse(t.isAlive()); org.testng.Assert.assertEquals(error.get(), null); } - private KafkaMonitor kafkaMonitor() throws Exception { + private XinfraMonitor kafkaMonitor() throws Exception { FakeService.clearCounters(); Map config = new HashMap<>(); Map fakeServiceConfig = new HashMap<>(); - fakeServiceConfig.put(KafkaMonitor.CLASS_NAME_CONFIG, FakeService.class.getName()); + fakeServiceConfig.put(XinfraMonitor.CLASS_NAME_CONFIG, FakeService.class.getName()); config.put("fake-service", fakeServiceConfig); - return new KafkaMonitor(config); + return new XinfraMonitor(config); } From 8c1ea3a7be64ac5b5dbaf4aa10a1d6848c7afe7f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 1 Jun 2020 17:18:07 -0700 Subject: [PATCH 119/192] Ensure KafkaMetricsReporterSerivce carries the correct constructor parameters. Ensure KafkaMetricsReporterSerivce carries the correct constructor parameters. Abstract out the helper methods with constructorContainsClass The way Service Instantiations are done within Kafka Monitor will be updated with https://github.com/linkedin/kafka-monitor/pull/257. testing: produce.treat.zero.throughput.as.unavailable = true topic = kafka-monitor-topic1 zookeeper.connect = localhost:2181 (com.linkedin.kmf.services.configs.ProduceServiceConfig) [2020-05-29 14:34:57,457] INFO produce-service/ProduceService is initialized. (com.linkedin.kmf.services.ProduceService) [2020-05-29 14:34:57,468] INFO KafkaMetricsReporterServiceConfig values: bootstrap.servers = localhost:9092 report.interval.sec = 3 report.kafka.topic.replication.factor = 1 report.metrics.list = [kmf.services:type=produce-service,name=*:produce-availability-avg, kmf.services:type=consume-service,name=*:consume-availability-avg, kmf.services:type=produce-service,name=*:records-produced-total, kmf.services:type=consume-service,name=*:records-consumed-total, kmf.services:type=consume-service,name=*:records-lost-total, kmf.services:type=consume-service,name=*:records-duplicated-total, kmf.services:type=consume-service,name=*:records-delay-ms-avg, kmf.services:type=produce-service,name=*:records-produced-rate, kmf.services:type=produce-service,name=*:produce-error-rate, kmf.services:type=consume-service,name=*:consume-error-rate] topic = kafka-monitor-topic1 zookeeper.connect = localhost:2181 (com.linkedin.kmf.services.configs.KafkaMetricsReporterServiceConfig) [2020-05-29 14:34:57,694] INFO CreateTopicsResult: {kafka-monitor-topic1=KafkaFuture{value=null,exception=null,done=true}}. (com.linkedin.kmf.common.Utils) [2020-05-29 14:34:57,694] INFO Created monitoring topic kafka-monitor-topic1 in cluster with 1 partitions and replication factor of 1. (com.linkedin.kmf.common.Utils) [2020-05-29 14:34:57,694] INFO Completed the topic creation if it doesn't exist for kafka-monitor-topic1. (com.linkedin.kmf.common.Utils) [2020-05-29 14:34:57,706] INFO produce-service/ProduceService started (com.linkedin.kmf.services.ProduceService) [2020-05-29 14:34:57,707] INFO reporter-kafka-service/KafkaMetricsReporterService has started. (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:34:57,707] INFO Xinfra Monitor (KafkaMonitor) started. (com.linkedin.kmf.KafkaMonitor) [2020-05-29 14:35:00,722] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "0.4514446227929374", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "27.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:35:03,708] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "0.9363765571440515", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "56.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:35:06,708] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "1.4045648357160772", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "84.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:35:09,709] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "1.8894425308497476", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "113.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:35:12,712] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "2.3742246150244948", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "142.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) [2020-05-29 14:35:15,709] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "2.842571691330156", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "170.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) f[2020-05-29 14:35:18,712] INFO Kafka Metrics Reporter sending metrics = { "kmf.services:name=produce-service,type=produce-service:records-produced-rate" : "3.3273140716960943", "kmf.services:name=produce-service,type=produce-service:records-produced-total" : "199.0", "kmf.services:name=produce-service,type=produce-service:produce-availability-avg" : "1.0", "kmf.services:name=produce-service,type=produce-service:produce-error-rate" : "0.0" } (com.linkedin.kmf.services.KafkaMetricsReporterService) Signed-off-by: Andrew Choi --- .../java/com/linkedin/kmf/XinfraMonitor.java | 17 ++++++++++++---- .../services/KafkaMetricsReporterService.java | 20 ++++++++++--------- 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/XinfraMonitor.java b/src/main/java/com/linkedin/kmf/XinfraMonitor.java index 626bf105..b4e08b1a 100644 --- a/src/main/java/com/linkedin/kmf/XinfraMonitor.java +++ b/src/main/java/com/linkedin/kmf/XinfraMonitor.java @@ -27,6 +27,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -60,7 +61,7 @@ public class XinfraMonitor { * For example, if there are 10 clusters to be monitored, then this Constructor will create 10 * num_apps_per_cluster * and 10 * num_services_per_cluster. * @param allClusterProps the properties of ALL kafka clusters for which apps and services need to be appended. - * @throws Exception + * @throws Exception when exception occurs while assigning Apps and Services */ @SuppressWarnings({"rawtypes", "unchecked"}) public XinfraMonitor(Map allClusterProps) throws Exception { @@ -80,7 +81,8 @@ public XinfraMonitor(Map allClusterProps) throws Exception { _apps.put(name, clusterApp); } else if (Service.class.isAssignableFrom(aClass)) { Constructor[] constructors = Class.forName(className).getConstructors(); - if (this.constructorContainsFuture(constructors)) { + if (this.constructorContainsClass(constructors, CompletableFuture.class)) { + // for ConsumeService public constructor CompletableFuture completableFuture = new CompletableFuture<>(); completableFuture.complete(null); ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); @@ -88,6 +90,13 @@ public XinfraMonitor(Map allClusterProps) throws Exception { .getConstructor(String.class, CompletableFuture.class, ConsumerFactory.class) .newInstance(name, completableFuture, consumerFactory); _services.put(name, service); + } else if (this.constructorContainsClass(constructors, AdminClient.class)) { + // for KafkaMetricsReporterService constructor + AdminClient adminClient = AdminClient.create(props); + Service service = (Service) Class.forName(className) + .getConstructor(Map.class, String.class, AdminClient.class) + .newInstance(props, name, adminClient); + _services.put(name, service); } else { Service service = (Service) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); _services.put(name, service); @@ -105,9 +114,9 @@ public XinfraMonitor(Map allClusterProps) throws Exception { (config, now) -> _offlineRunnables.size()); } - private boolean constructorContainsFuture(Constructor[] constructors) { + private boolean constructorContainsClass(Constructor[] constructors, Class classObject) { for (int n = 0; n < constructors[0].getParameterTypes().length; ++n) { - if (constructors[0].getParameterTypes()[n].equals(CompletableFuture.class)) { + if (constructors[0].getParameterTypes()[n].equals(classObject)) { return true; } } diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java index 1e613fee..9bd8ac16 100644 --- a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterService.java @@ -28,8 +28,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class KafkaMetricsReporterService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricsReporterService.class); + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaMetricsReporterService.class); private static final String METRICS_PRODUCER_ID = "kafka-metrics-reporter-id"; private final String _name; private final List _metricsNames; @@ -49,9 +50,10 @@ public KafkaMetricsReporterService(Map props, String name, Admin _brokerList = config.getString(KafkaMetricsReporterServiceConfig.BOOTSTRAP_SERVERS_CONFIG); initializeProducer(); _topic = config.getString(KafkaMetricsReporterServiceConfig.TOPIC_CONFIG); + Integer rf = config.getInt(KafkaMetricsReporterServiceConfig.TOPIC_REPLICATION_FACTOR); Utils.createTopicIfNotExists( _topic, - config.getShort(KafkaMetricsReporterServiceConfig.TOPIC_REPLICATION_FACTOR), + rf.shortValue(), 0, // parameter is set to 0 here since no matter the number of nodes, the topic partition number should be set to zero. 1, // fixed partition count 1 new Properties(), @@ -65,17 +67,17 @@ public synchronized void start() { try { reportMetrics(); } catch (Exception e) { - LOG.error(_name + "/KafkaMetricsReporterService failed to report metrics", e); + LOGGER.error(_name + "/KafkaMetricsReporterService failed to report metrics.", e); } }, _reportIntervalSec, _reportIntervalSec, TimeUnit.SECONDS); - LOG.info("{}/KafkaMetricsReporterService started", _name); + LOGGER.info("{}/KafkaMetricsReporterService has started.", _name); } @Override public synchronized void stop() { _executor.shutdown(); _producer.close(); - LOG.info("{}/KafkaMetricsReporterService stopped", _name); + LOGGER.info("{}/KafkaMetricsReporterService stopped.", _name); } @Override @@ -88,9 +90,9 @@ public void awaitShutdown() { try { _executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.info("Thread interrupted when waiting for {}/KafkaMetricsReporterService to shutdown", _name); + LOGGER.info("Thread interrupted when waiting for {}/KafkaMetricsReporterService to shutdown", _name); } - LOG.info("{}/KafkaMetricsReporterService shutdown completed", _name); + LOGGER.info("{}/KafkaMetricsReporterService shutdown completed", _name); } @@ -122,10 +124,10 @@ private void reportMetrics() { } } try { - LOG.debug("Kafka Metrics Reporter sending metrics = " + _parser.writerWithDefaultPrettyPrinter().writeValueAsString(metrics)); + LOGGER.info("Kafka Metrics Reporter sending metrics = " + _parser.writerWithDefaultPrettyPrinter().writeValueAsString(metrics)); _producer.send(new ProducerRecord<>(_topic, _parser.writeValueAsString(metrics))); } catch (JsonProcessingException e) { - LOG.warn("unsupported json format: " + metrics, e); + LOGGER.warn("unsupported json format: " + metrics, e); } } } From 2fc22fd4d9bda61bda8a3196cc3d94e60855e091 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 1 Jun 2020 18:03:49 -0700 Subject: [PATCH 120/192] Fallback to regular constructor parameters if the constructor doesn't include AdminClient. #253 Fallback to regular constructor parameters if the constructor doesn't include AdminClient. #253 --- .../kmf/services/ConsumerFactoryImpl.java | 24 ++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java index f08e2472..e3427d4e 100644 --- a/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java +++ b/src/main/java/com/linkedin/kmf/services/ConsumerFactoryImpl.java @@ -81,10 +81,28 @@ public ConsumerFactoryImpl(Map props) throws Exception { props.forEach(consumerProps::putIfAbsent); } - _baseConsumer = (KMBaseConsumer) Class.forName(consumerClassName) - .getConstructor(String.class, Properties.class, AdminClient.class) - .newInstance(_topic, consumerProps, adminClient()); + java.lang.reflect.Constructor constructor = adminClientConstructorIfExists(consumerClassName); + if (constructor != null) { + _baseConsumer = (KMBaseConsumer) constructor + .newInstance(_topic, consumerProps, adminClient()); + } else { + _baseConsumer = (KMBaseConsumer) Class.forName(consumerClassName) + .getConstructor(String.class, Properties.class) + .newInstance(_topic, consumerProps); + } + } + private static java.lang.reflect.Constructor adminClientConstructorIfExists(String consumerClassName) + throws ClassNotFoundException { + try { + return Class.forName(consumerClassName).getConstructor(String.class, Properties.class, AdminClient.class); + } catch (java.lang.NoSuchMethodException noSuchMethodException) { + LOG.info(consumerClassName + + " does not provide a constructor with signature (Ljava/lang/String;Ljava/util/Properties;Lorg/apache/kafka/clients/admin/AdminClient;)V - falling back to (Ljava/util/Properties;)V"); + return null; + } catch (ClassNotFoundException e) { + throw new ClassNotFoundException("The class was not found: ", e); + } } @Override From 10045c5002ff3fd20bffad7a89b2f94ab118bf13 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 1 Jun 2020 23:05:44 -0700 Subject: [PATCH 121/192] Service Factory + Individual Services' Factory Implementations (#257) Service Factory + Individual Services' Factory Implementations Because of the early design, architectural decisions of Xinfra Monitor, the KafkaMonitor class enforces individual Xinfra Monitor Services to carry the same set of constructor parameters. This is a large caveat as not all Service classes don't need the identical constructor arguments. This is not enforced the codebase. It is partially and only silently enforced by convention. Unmaintainable code + non-robust code. Non-evolvable code: each Service's constructor implementations aren't allowed to be evolved independently. 3. Change(s) in one Service's constructor absolutely forces the other Services' constructors to also change its parameters, unfortunately. Signed off by : Andrew Choi --- .../java/com/linkedin/kmf/XinfraMonitor.java | 48 +++++---------- .../linkedin/kmf/XinfraMonitorConstants.java | 30 ++++++++++ .../kmf/services/ConsumeServiceFactory.java | 39 +++++++++++++ .../DefaultMetricsReporterServiceFactory.java | 35 +++++++++++ ...GraphiteMetricsReporterServiceFactory.java | 36 ++++++++++++ .../kmf/services/JolokiaServiceFactory.java | 36 ++++++++++++ .../KafkaMetricsReporterServiceFactory.java | 41 +++++++++++++ ...iClusterTopicManagementServiceFactory.java | 36 ++++++++++++ .../kmf/services/ProduceServiceFactory.java | 34 +++++++++++ .../linkedin/kmf/services/ServiceFactory.java | 17 ++++++ ...SignalFxMetricsReporterServiceFactory.java | 37 ++++++++++++ .../StatsdMetricsReporterServiceFactory.java | 37 ++++++++++++ .../TopicManagementServiceFactory.java | 37 ++++++++++++ .../com/linkedin/kmf/KafkaMonitorTest.java | 58 ++++++++++++++----- 14 files changed, 472 insertions(+), 49 deletions(-) create mode 100644 src/main/java/com/linkedin/kmf/XinfraMonitorConstants.java create mode 100644 src/main/java/com/linkedin/kmf/services/ConsumeServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/JolokiaServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/ProduceServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/ServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterServiceFactory.java create mode 100644 src/main/java/com/linkedin/kmf/services/TopicManagementServiceFactory.java diff --git a/src/main/java/com/linkedin/kmf/XinfraMonitor.java b/src/main/java/com/linkedin/kmf/XinfraMonitor.java index b4e08b1a..e80246ca 100644 --- a/src/main/java/com/linkedin/kmf/XinfraMonitor.java +++ b/src/main/java/com/linkedin/kmf/XinfraMonitor.java @@ -7,27 +7,25 @@ * 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. */ + package com.linkedin.kmf; import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.kmf.apps.App; -import com.linkedin.kmf.services.ConsumerFactory; -import com.linkedin.kmf.services.ConsumerFactoryImpl; import com.linkedin.kmf.services.Service; +import com.linkedin.kmf.services.ServiceFactory; import java.io.BufferedReader; import java.io.FileReader; import java.lang.reflect.Constructor; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -42,9 +40,6 @@ */ public class XinfraMonitor { private static final Logger LOG = LoggerFactory.getLogger(XinfraMonitor.class); - public static final String CLASS_NAME_CONFIG = "class.name"; - private static final String METRIC_GROUP_NAME = "kafka-monitor"; - private static final String JMX_PREFIX = "kmf"; /** This is concurrent because healthCheck() can modify this map, but awaitShutdown() can be called at any time by * a different thread. @@ -63,7 +58,8 @@ public class XinfraMonitor { * @param allClusterProps the properties of ALL kafka clusters for which apps and services need to be appended. * @throws Exception when exception occurs while assigning Apps and Services */ - @SuppressWarnings({"rawtypes", "unchecked"}) + + @SuppressWarnings({"rawtypes"}) public XinfraMonitor(Map allClusterProps) throws Exception { _apps = new ConcurrentHashMap<>(); _services = new ConcurrentHashMap<>(); @@ -71,36 +67,20 @@ public XinfraMonitor(Map allClusterProps) throws Exception { for (Map.Entry clusterProperty : allClusterProps.entrySet()) { String name = clusterProperty.getKey(); Map props = clusterProperty.getValue(); - if (!props.containsKey(CLASS_NAME_CONFIG)) - throw new IllegalArgumentException(name + " is not configured with " + CLASS_NAME_CONFIG); - String className = (String) props.get(CLASS_NAME_CONFIG); + if (!props.containsKey(XinfraMonitorConstants.CLASS_NAME_CONFIG)) + throw new IllegalArgumentException(name + " is not configured with " + XinfraMonitorConstants.CLASS_NAME_CONFIG); + String className = (String) props.get(XinfraMonitorConstants.CLASS_NAME_CONFIG); Class aClass = Class.forName(className); if (App.class.isAssignableFrom(aClass)) { App clusterApp = (App) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); _apps.put(name, clusterApp); } else if (Service.class.isAssignableFrom(aClass)) { - Constructor[] constructors = Class.forName(className).getConstructors(); - if (this.constructorContainsClass(constructors, CompletableFuture.class)) { - // for ConsumeService public constructor - CompletableFuture completableFuture = new CompletableFuture<>(); - completableFuture.complete(null); - ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(props); - Service service = (Service) Class.forName(className) - .getConstructor(String.class, CompletableFuture.class, ConsumerFactory.class) - .newInstance(name, completableFuture, consumerFactory); - _services.put(name, service); - } else if (this.constructorContainsClass(constructors, AdminClient.class)) { - // for KafkaMetricsReporterService constructor - AdminClient adminClient = AdminClient.create(props); - Service service = (Service) Class.forName(className) - .getConstructor(Map.class, String.class, AdminClient.class) - .newInstance(props, name, adminClient); - _services.put(name, service); - } else { - Service service = (Service) Class.forName(className).getConstructor(Map.class, String.class).newInstance(props, name); - _services.put(name, service); - } + ServiceFactory serviceFactory = (ServiceFactory) Class.forName(className + XinfraMonitorConstants.FACTORY) + .getConstructor(Map.class, String.class) + .newInstance(props, name); + Service service = serviceFactory.createService(); + _services.put(name, service); } else { throw new IllegalArgumentException(className + " should implement either " + App.class.getSimpleName() + " or " + Service.class.getSimpleName()); } @@ -108,9 +88,9 @@ public XinfraMonitor(Map allClusterProps) throws Exception { _executor = Executors.newSingleThreadScheduledExecutor(); _offlineRunnables = new ConcurrentHashMap<>(); List reporters = new ArrayList<>(); - reporters.add(new JmxReporter(JMX_PREFIX)); + reporters.add(new JmxReporter(XinfraMonitorConstants.JMX_PREFIX)); Metrics metrics = new Metrics(new MetricConfig(), reporters, new SystemTime()); - metrics.addMetric(metrics.metricName("offline-runnable-count", METRIC_GROUP_NAME, "The number of Service/App that are not fully running"), + metrics.addMetric(metrics.metricName("offline-runnable-count", XinfraMonitorConstants.METRIC_GROUP_NAME, "The number of Service/App that are not fully running"), (config, now) -> _offlineRunnables.size()); } diff --git a/src/main/java/com/linkedin/kmf/XinfraMonitorConstants.java b/src/main/java/com/linkedin/kmf/XinfraMonitorConstants.java new file mode 100644 index 00000000..3d78b08f --- /dev/null +++ b/src/main/java/com/linkedin/kmf/XinfraMonitorConstants.java @@ -0,0 +1,30 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf; + +/** + * Constant variables in Xinfra Monitor repo. + */ +public class XinfraMonitorConstants { + + public XinfraMonitorConstants() { + } + + static final String FACTORY = "Factory"; + + static final String CLASS_NAME_CONFIG = "class.name"; + + static final String METRIC_GROUP_NAME = "kafka-monitor"; + + static final String JMX_PREFIX = "kmf"; + + +} diff --git a/src/main/java/com/linkedin/kmf/services/ConsumeServiceFactory.java b/src/main/java/com/linkedin/kmf/services/ConsumeServiceFactory.java new file mode 100644 index 00000000..fcf3af16 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ConsumeServiceFactory.java @@ -0,0 +1,39 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + + +/** + * Factory that constructs the ConsumeService. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ConsumeServiceFactory implements ServiceFactory { + private final Map _props; + private final String _name; + + public ConsumeServiceFactory(Map props, String name) { + _props = props; + _name = name; + } + + @Override + public Service createService() throws Exception { + + CompletableFuture topicPartitionResult = new CompletableFuture<>(); + topicPartitionResult.complete(null); + ConsumerFactoryImpl consumerFactory = new ConsumerFactoryImpl(_props); + + return new ConsumeService(_name, topicPartitionResult, consumerFactory); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterServiceFactory.java b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterServiceFactory.java new file mode 100644 index 00000000..29f89618 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/DefaultMetricsReporterServiceFactory.java @@ -0,0 +1,35 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class which instantiates a DefaultMetricsReporterService. + */ +@SuppressWarnings("rawtypes") +public class DefaultMetricsReporterServiceFactory implements ServiceFactory { + private final Map _properties; + private final String _serviceName; + + public DefaultMetricsReporterServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new DefaultMetricsReporterService(_properties, _serviceName); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterServiceFactory.java b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterServiceFactory.java new file mode 100644 index 00000000..2a34e7cd --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/GraphiteMetricsReporterServiceFactory.java @@ -0,0 +1,36 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class which instantiates a GraphiteMetricsReporterServiceFactory service. + */ +@SuppressWarnings("rawtypes") +public class GraphiteMetricsReporterServiceFactory implements ServiceFactory { + + private final Map _properties; + private final String _serviceName; + + public GraphiteMetricsReporterServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new GraphiteMetricsReporterService(_properties, _serviceName); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/JolokiaServiceFactory.java b/src/main/java/com/linkedin/kmf/services/JolokiaServiceFactory.java new file mode 100644 index 00000000..540cdae5 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/JolokiaServiceFactory.java @@ -0,0 +1,36 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class which instantiates a JolokiaService service. + */ +@SuppressWarnings("rawtypes") +public class JolokiaServiceFactory implements ServiceFactory { + + private final Map _properties; + private final String _serviceName; + + public JolokiaServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new JolokiaService(_properties, _serviceName); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterServiceFactory.java b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterServiceFactory.java new file mode 100644 index 00000000..b72c2a4e --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/KafkaMetricsReporterServiceFactory.java @@ -0,0 +1,41 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; +import org.apache.kafka.clients.admin.AdminClient; + + +/** + * Factory class which instantiates a KafkaMetricsReporterService service object. + */ +@SuppressWarnings("rawtypes") +public class KafkaMetricsReporterServiceFactory implements ServiceFactory { + + private final Map _properties; + private final String _serviceName; + + public KafkaMetricsReporterServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + + AdminClient adminClient = AdminClient.create(_properties); + + return new KafkaMetricsReporterService(_properties, _serviceName, adminClient); + + } +} diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceFactory.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceFactory.java new file mode 100644 index 00000000..bf802a6d --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceFactory.java @@ -0,0 +1,36 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory which instantiates a MultiClusterTopicManagementService service object. + */ +@SuppressWarnings("rawtypes") +public class MultiClusterTopicManagementServiceFactory implements ServiceFactory { + + private final Map _properties; + private final String _serviceName; + + public MultiClusterTopicManagementServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new MultiClusterTopicManagementService(_properties, _serviceName); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/ProduceServiceFactory.java b/src/main/java/com/linkedin/kmf/services/ProduceServiceFactory.java new file mode 100644 index 00000000..9102b700 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ProduceServiceFactory.java @@ -0,0 +1,34 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory that constructs the ProduceService + */ +@SuppressWarnings("rawtypes") +public class ProduceServiceFactory implements ServiceFactory { + private final Map _props; + private final String _name; + + public ProduceServiceFactory(Map props, String name) { + _props = props; + _name = name; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new ProduceService(_props, _name); + } +} diff --git a/src/main/java/com/linkedin/kmf/services/ServiceFactory.java b/src/main/java/com/linkedin/kmf/services/ServiceFactory.java new file mode 100644 index 00000000..0f5c9411 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/ServiceFactory.java @@ -0,0 +1,17 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +public interface ServiceFactory { + + Service createService() throws Exception; + +} diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterServiceFactory.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterServiceFactory.java new file mode 100644 index 00000000..b62d7d7c --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterServiceFactory.java @@ -0,0 +1,37 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class which instantiates a SignalFxMetricsReporterServiceFactory service. + */ +@SuppressWarnings("rawtypes") +public class SignalFxMetricsReporterServiceFactory implements ServiceFactory { + + private final Map _properties; + private final String _serviceName; + + public SignalFxMetricsReporterServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + return new SignalFxMetricsReporterService(_properties, _serviceName); + } +} + diff --git a/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterServiceFactory.java b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterServiceFactory.java new file mode 100644 index 00000000..857c00e6 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/StatsdMetricsReporterServiceFactory.java @@ -0,0 +1,37 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class that constructs the StatsdMetricsReporterService. + */ +@SuppressWarnings("rawtypes") +public class StatsdMetricsReporterServiceFactory implements ServiceFactory { + private final Map _properties; + private final String _name; + + public StatsdMetricsReporterServiceFactory(Map properties, String name) { + + _properties = properties; + _name = name; + } + + @Override + public Service createService() throws Exception { + + //noinspection unchecked + return new StatsdMetricsReporterService(_properties, _name); + + } +} diff --git a/src/main/java/com/linkedin/kmf/services/TopicManagementServiceFactory.java b/src/main/java/com/linkedin/kmf/services/TopicManagementServiceFactory.java new file mode 100644 index 00000000..2dfbdf54 --- /dev/null +++ b/src/main/java/com/linkedin/kmf/services/TopicManagementServiceFactory.java @@ -0,0 +1,37 @@ +/** + * Copyright 2020 LinkedIn Corp. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this + * file except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + */ + +package com.linkedin.kmf.services; + +import java.util.Map; + + +/** + * Factory class which constructs the TopicManagementService. + */ +@SuppressWarnings("rawtypes") +public class TopicManagementServiceFactory implements ServiceFactory { + private final Map _properties; + private final String _serviceName; + + public TopicManagementServiceFactory(Map properties, String serviceName) { + + _properties = properties; + _serviceName = serviceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + + return new TopicManagementService(_properties, _serviceName); + + } +} diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java index fa6e65c0..b794aaed 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java @@ -11,6 +11,7 @@ package com.linkedin.kmf; import com.linkedin.kmf.services.Service; +import com.linkedin.kmf.services.ServiceFactory; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -27,22 +28,24 @@ public void lifecycleTest() throws Exception { XinfraMonitor xinfraMonitor = kafkaMonitor(); /* Nothing should be started */ - org.testng.Assert.assertEquals(FakeService.startCount.get(), 0); - org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); + org.testng.Assert.assertEquals(FakeService.START_COUNT.get(), 0); + org.testng.Assert.assertEquals(FakeService.STOP_COUNT.get(), 0); /* Should accept but ignore start because start has not been called */ + xinfraMonitor.stop(); - org.testng.Assert.assertEquals(FakeService.stopCount.get(), 0); + org.testng.Assert.assertEquals(FakeService.STOP_COUNT.get(), 0); /* Should start */ xinfraMonitor.start(); - org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); + org.testng.Assert.assertEquals(FakeService.START_COUNT.get(), 1); /* Should allow start to be called more than once */ xinfraMonitor.stop(); xinfraMonitor.stop(); - org.testng.Assert.assertEquals(FakeService.startCount.get(), 1); - org.testng.Assert.assertEquals(FakeService.stopCount.get(), 1); + org.testng.Assert.assertEquals(FakeService.START_COUNT.get(), 1); + org.testng.Assert.assertEquals(FakeService.STOP_COUNT.get(), 1); + /* Should be allowed to shutdown more than once. */ xinfraMonitor.awaitShutdown(); @@ -78,16 +81,41 @@ private XinfraMonitor kafkaMonitor() throws Exception { FakeService.clearCounters(); Map config = new HashMap<>(); Map fakeServiceConfig = new HashMap<>(); - fakeServiceConfig.put(XinfraMonitor.CLASS_NAME_CONFIG, FakeService.class.getName()); + + fakeServiceConfig.put(XinfraMonitorConstants.CLASS_NAME_CONFIG, FakeService.class.getName()); config.put("fake-service", fakeServiceConfig); return new XinfraMonitor(config); + } + /** + * Factory class which instantiates a new FakeService service object. + */ + @SuppressWarnings("rawtypes") + static final class FakeServiceFactory implements ServiceFactory { + + private final Map _config; + private final String _serviceInstanceName; + + public FakeServiceFactory(Map config, String serviceInstanceName) { + + this._config = config; + this._serviceInstanceName = serviceInstanceName; + } + + @SuppressWarnings("unchecked") + @Override + public Service createService() throws Exception { + + return new KafkaMonitorTest.FakeService(_config, _serviceInstanceName); + + } + } static final class FakeService implements Service { - private static AtomicInteger startCount = new AtomicInteger(); - private static AtomicInteger stopCount = new AtomicInteger(); + private static final AtomicInteger START_COUNT = new AtomicInteger(); + private static final AtomicInteger STOP_COUNT = new AtomicInteger(); private final AtomicBoolean _isRunning = new AtomicBoolean(); /** required */ @@ -96,20 +124,20 @@ public FakeService(Map config, String serviceInstanceName) { } private static void clearCounters() { - startCount.set(0); - stopCount.set(0); + START_COUNT.set(0); + STOP_COUNT.set(0); } @Override public void start() { _isRunning.compareAndSet(false, true); - startCount.incrementAndGet(); + START_COUNT.incrementAndGet(); } @Override public synchronized void stop() { _isRunning.compareAndSet(true, false); - stopCount.incrementAndGet(); + STOP_COUNT.incrementAndGet(); notifyAll(); } @@ -121,9 +149,9 @@ public boolean isRunning() { @Override public synchronized void awaitShutdown() { try { - if (stopCount.get() == 0) { + if (STOP_COUNT.get() == 0) { wait(3_000); - if (stopCount.get() == 0) { + if (STOP_COUNT.get() == 0) { throw new IllegalStateException("Never notified."); } } From d9634a9441afc0abe2ed4116a732f77ce63ef7c6 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Tue, 2 Jun 2020 20:58:14 -0700 Subject: [PATCH 122/192] xinfra rebrading (#259) Signed-off-by: Andrew Choi --- README.md | 2 +- config/kafka-monitor.properties | 2 +- src/main/java/com/linkedin/kmf/XinfraMonitor.java | 4 ++-- src/main/java/com/linkedin/kmf/common/Utils.java | 2 +- .../{KafkaMonitorTest.java => XinfraMonitorTest.java} | 10 +++++----- 5 files changed, 10 insertions(+), 10 deletions(-) rename src/test/java/com/linkedin/kmf/{KafkaMonitorTest.java => XinfraMonitorTest.java} (94%) diff --git a/README.md b/README.md index 95b1ef63..07bbfbe7 100644 --- a/README.md +++ b/README.md @@ -90,7 +90,7 @@ $ cd kafka-monitor $ ./gradlew jar ``` -### Start KafkaMonitor to run tests/services specified in the config file +### Start XinfraMonitor to run tests/services specified in the config file ``` $ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties ``` diff --git a/config/kafka-monitor.properties b/config/kafka-monitor.properties index 79f8b816..7553664a 100644 --- a/config/kafka-monitor.properties +++ b/config/kafka-monitor.properties @@ -6,7 +6,7 @@ # 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. -# This properties file specifies the tests/services that KafkaMonitor +# This properties file specifies the tests/services that XinfraMonitor # should instantiate and run, together with the key/value pairs used to # configure these tests/services. It should have the following format: # diff --git a/src/main/java/com/linkedin/kmf/XinfraMonitor.java b/src/main/java/com/linkedin/kmf/XinfraMonitor.java index e80246ca..8a3cad14 100644 --- a/src/main/java/com/linkedin/kmf/XinfraMonitor.java +++ b/src/main/java/com/linkedin/kmf/XinfraMonitor.java @@ -52,7 +52,7 @@ public class XinfraMonitor { private final AtomicBoolean _isRunning = new AtomicBoolean(false); /** - * KafkaMonitor constructor creates apps and services for each of the individual clusters (properties) that's passed in. + * XinfraMonitor constructor creates apps and services for each of the individual clusters (properties) that's passed in. * For example, if there are 10 clusters to be monitored, then this Constructor will create 10 * num_apps_per_cluster * and 10 * num_services_per_cluster. * @param allClusterProps the properties of ALL kafka clusters for which apps and services need to be appended. @@ -185,7 +185,7 @@ public static void main(String[] args) throws Exception { Map props = new ObjectMapper().readValue(buffer.toString(), Map.class); XinfraMonitor xinfraMonitor = new XinfraMonitor(props); xinfraMonitor.start(); - LOG.info("Xinfra Monitor (KafkaMonitor) started."); + LOG.info("Xinfra Monitor has started."); xinfraMonitor.awaitShutdown(); } diff --git a/src/main/java/com/linkedin/kmf/common/Utils.java b/src/main/java/com/linkedin/kmf/common/Utils.java index 8aa0287f..4afaa966 100644 --- a/src/main/java/com/linkedin/kmf/common/Utils.java +++ b/src/main/java/com/linkedin/kmf/common/Utils.java @@ -130,7 +130,7 @@ private static int getBrokerCount(AdminClient adminClient) throws ExecutionExcep /** * @param timestamp time in Ms when this message is generated * @param topic topic this message is sent to - * @param idx index is consecutive numbers used by KafkaMonitor to determine duplicate or lost messages + * @param idx index is consecutive numbers used by XinfraMonitor to determine duplicate or lost messages * @param msgSize size of the message * @return string that encodes the above fields */ diff --git a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java b/src/test/java/com/linkedin/kmf/XinfraMonitorTest.java similarity index 94% rename from src/test/java/com/linkedin/kmf/KafkaMonitorTest.java rename to src/test/java/com/linkedin/kmf/XinfraMonitorTest.java index b794aaed..1844738b 100644 --- a/src/test/java/com/linkedin/kmf/KafkaMonitorTest.java +++ b/src/test/java/com/linkedin/kmf/XinfraMonitorTest.java @@ -21,11 +21,11 @@ @Test -public class KafkaMonitorTest { +public class XinfraMonitorTest { @Test public void lifecycleTest() throws Exception { - XinfraMonitor xinfraMonitor = kafkaMonitor(); + XinfraMonitor xinfraMonitor = xinfraMonitor(); /* Nothing should be started */ org.testng.Assert.assertEquals(FakeService.START_COUNT.get(), 0); @@ -54,7 +54,7 @@ public void lifecycleTest() throws Exception { @Test public void awaitShutdownOtherThread() throws Exception { - final XinfraMonitor xinfraMonitor = kafkaMonitor(); + final XinfraMonitor xinfraMonitor = xinfraMonitor(); final AtomicReference error = new AtomicReference<>(); Thread t = new Thread("test awaitshutdown thread") { @@ -77,7 +77,7 @@ public void run() { org.testng.Assert.assertEquals(error.get(), null); } - private XinfraMonitor kafkaMonitor() throws Exception { + private XinfraMonitor xinfraMonitor() throws Exception { FakeService.clearCounters(); Map config = new HashMap<>(); Map fakeServiceConfig = new HashMap<>(); @@ -107,7 +107,7 @@ public FakeServiceFactory(Map config, String serviceInstanceName) { @Override public Service createService() throws Exception { - return new KafkaMonitorTest.FakeService(_config, _serviceInstanceName); + return new XinfraMonitorTest.FakeService(_config, _serviceInstanceName); } } From 9b15dc0033c217d4150992ab3eb6548543b4afb4 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 3 Jun 2020 13:23:15 -0700 Subject: [PATCH 123/192] Xinfra Monitor Rebranding Migration Continuation Part III Signed-off-by: Andrew Choi Xinfra Monitor Rebranding Migration Continuation Part III Signed-off-by: Andrew Choi --- .travis.yml | 2 +- README.md | 14 +++++++------- bin/windows/kafka-monitor-start.bat | 2 +- bin/windows/kmf-run-class.bat | 4 ++-- ...ka-monitor-start.sh => xinfra-monitor-start.sh} | 0 ...onitor.properties => xinfra-monitor.properties} | 8 ++++---- docker/Dockerfile | 2 +- docker/kafka-monitor-docker-entry.sh | 2 +- src/main/java/com/linkedin/kmf/XinfraMonitor.java | 2 +- .../StatsdMetricsReporterServiceConfig.java | 2 +- 10 files changed, 19 insertions(+), 19 deletions(-) rename bin/{kafka-monitor-start.sh => xinfra-monitor-start.sh} (100%) rename config/{kafka-monitor.properties => xinfra-monitor.properties} (97%) diff --git a/.travis.yml b/.travis.yml index bb6c28bb..d866cfeb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,7 +4,7 @@ jdk: oraclejdk8 env: global: - # these secure values are encrypted for linkedin/kafka-monitor. for use in forked repos, set env vars in your own build - they should win over these (as these wont be decodable in forked builds) + # these secure values are encrypted for linkedin/xinfra-monitor. for use in forked repos, set env vars in your own build - they should win over these (as these wont be decodable in forked builds) - secure: ghCdYRfSGI7BtA0rB0UfzEJWszKwvnSy1Y0LBSWn3wdo3e0ibHdofwfeQ5YAczXun+RpGvaUwJRgkSDOsL6DFF8N2hODNkzR+DFNRJv5CFOYuCPJZwPpw/Sr7blCkpZleWmJBW7sqRPa7Yyu5/QpdezRM4tHP/RCxjr9IKgQwOknYyVGUf81Wa3Y+MF5iqR434WItfFpHR7uU5sis592iR4aKS4bcEQ9DCqcrCsrr3RFAft6aM5NZjAhO7tXe3vu5IrtqbB/MFNLrnRO0T6kXcifVqVrHQ/ePzl2iyPIdKygiDgK9VOYg6TfT5maHLwoCK511qnhMlitsy/0qx8CsF33rS0hXS5+EDSJ3qoB38Rqgi2K08uGjWgwwXtUg+2CkyEDA2gpEDIELQSFvG0WEHmAim9Azfxiiyx+HwNJ0zkKeOPUXU3uLDeOGXsUB6vxqjrWvAGtUqCZ5g2qEI8geGH3Zopmg9j665FywawV0OAehj4I2Tn0rBDORbIsu3XWgcZIbq/D8sb4Xhb65CB4wSvcco5g25zQzXruDk6MjB76bthtnIbO9hvIVwADxx2MdWXOX873ApjRoffGtRev+ugfEZCEuWxBvtgJ2owCSdTOm8TIjAwjsc4eBuLqEWQplwnoGhRA12zch4t/itOGv0ABoT9/tVjhiLwtTzmHTFc= - secure: I88NyZbbsV4j5a14k1ModOLsQygs7cXjCY98QQcQNRV0FKvvk/ka7xEvwiOFPWFOFy7TUf9O2VjNA8M4oWGqop9MwYNysAccuX6y/VMAldOPlQ23PG7C+Tr44u2YgvbJCdjtmpVKh842yrU8nlHcfLKBGMaqRxjBg4kHNr0lyTGIU83UBsU26yFiUBuE0EwdWXcrlX+ZvfS523BezquWnFW+75R4z2e0i++/vGqKnlBsqXzzxR61aFnYuSN0CDj+XbG1hsc1dcNJbjLVfO8zOSd25oDYVEX+dfBEfD+4LL/3YSjM8EJvUZGghiaDR+K9nm26B+nfvWNOWH8PuqM94t65vueQ4qcUIUXqwhbIf5GixHGTOIj7QOjckPHZ4N4hYkpYMLwPRO+I5VuBEKlboCN90c3+OEEsBSUXSH/JIBI8mmiwkegPV89HtSRntFGFqYtL7Wg/GyJV7DCQSsacJDaM9ErX2z1sUd++QAQA18qdk1Ngl2hnBOWmTgmtefMo9u15iCxDZr+iMeeTA0IvsupyXeO6hN9xFuwRuXD/q40RIYcOgqFeYR9e/pRJ1A29PE4N6uqDe/JBA8oQFH0jCFbO2oEpVJlX9qgp29wWLSvm/93LRK3f8RLt8ZT1BVNEUnRqtkprbDVp2QlN/7fOz9MI9a00SAXv+siaz3oK++k= diff --git a/README.md b/README.md index 07bbfbe7..ee7a5975 100644 --- a/README.md +++ b/README.md @@ -45,8 +45,8 @@ Xinfra Monitor supports Apache Kafka 0.8 to 2.0:
  1. We advise advanced users to run Xinfra Monitor with -./bin/kafka-monitor-start.sh config/kafka-monitor.properties. The default -kafka-monitor.properties in the repo provides an simple example of how to +./bin/kafka-monitor-start.sh config/xinfra-monitor.properties. The default +xinfra-monitor.properties in the repo provides an simple example of how to monitor a single cluster. You probably need to change the value of zookeeper.connect and bootstrap.servers to point to your cluster.
  2. @@ -55,7 +55,7 @@ monitor a single cluster. You probably need to change the value of Config class for respective service, e.g. ProduceServiceConfig.java and ConsumeServiceConfig.java.
    -
  3. You can specify multiple SingleClusterMonitor in the kafka-monitor.properties to +
  4. You can specify multiple SingleClusterMonitor in the xinfra-monitor.properties to monitor multiple Kafka clusters in one Xinfra Monitor process. As another advanced use-case, you can point ProduceService and ConsumeService to two different Kafka clusters that are connected by MirrorMaker to monitor their end-to-end latency.

  5. @@ -92,16 +92,16 @@ $ ./gradlew jar ### Start XinfraMonitor to run tests/services specified in the config file ``` -$ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties +$ ./bin/kafka-monitor-start.sh config/xinfra-monitor.properties ``` ### Run Xinfra Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) -Edit `config/kafka-monitor.properties` to specify custom configurations for producer in the key/value map `produce.producer.props` in -`config/kafka-monitor.properties`. Similarly specify configurations for +Edit `config/xinfra-monitor.properties` to specify custom configurations for producer in the key/value map `produce.producer.props` in +`config/xinfra-monitor.properties`. Similarly specify configurations for consumer as well. The documentation for producer and consumer in the key/value maps can be found in the Apache Kafka wiki. ``` -$ ./bin/kafka-monitor-start.sh config/kafka-monitor.properties +$ ./bin/kafka-monitor-start.sh config/xinfra-monitor.properties ``` ### Run SingleClusterMonitor app to monitor kafka cluster diff --git a/bin/windows/kafka-monitor-start.bat b/bin/windows/kafka-monitor-start.bat index d8928ee2..e4531f91 100644 --- a/bin/windows/kafka-monitor-start.bat +++ b/bin/windows/kafka-monitor-start.bat @@ -15,7 +15,7 @@ popd IF [%1] EQU [] ( - echo USAGE: %0 config/kafka-monitor.properties + echo USAGE: %0 config/xinfra-monitor.properties EXIT /B 1 ) diff --git a/bin/windows/kmf-run-class.bat b/bin/windows/kmf-run-class.bat index 619f70d3..ca79a7b4 100644 --- a/bin/windows/kmf-run-class.bat +++ b/bin/windows/kmf-run-class.bat @@ -10,12 +10,12 @@ REM an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either expre setlocal enabledelayedexpansion IF [%1] EQU [] ( - echo USAGE: %0 com.linkedin.kmf.XinfraMonitor config/kafka-monitor.properties + echo USAGE: %0 com.linkedin.kmf.XinfraMonitor config/xinfra-monitor.properties EXIT /B 1 ) IF [%2] EQU [] ( - echo USAGE: %0 %1 config/kafka-monitor.properties + echo USAGE: %0 %1 config/xinfra-monitor.properties EXIT /B 1 ) diff --git a/bin/kafka-monitor-start.sh b/bin/xinfra-monitor-start.sh similarity index 100% rename from bin/kafka-monitor-start.sh rename to bin/xinfra-monitor-start.sh diff --git a/config/kafka-monitor.properties b/config/xinfra-monitor.properties similarity index 97% rename from config/kafka-monitor.properties rename to config/xinfra-monitor.properties index 7553664a..d36c1923 100644 --- a/config/kafka-monitor.properties +++ b/config/xinfra-monitor.properties @@ -45,7 +45,7 @@ { "single-cluster-monitor": { "class.name": "com.linkedin.kmf.apps.SingleClusterMonitor", - "topic": "kafka-monitor-topic", + "topic": "xinfra-monitor-topic", "zookeeper.connect": "localhost:2181", "bootstrap.servers": "localhost:9092", "request.timeout.ms": 9000, @@ -104,7 +104,7 @@ # Example produce-service to produce messages to cluster # "produce-service": { # "class.name": "com.linkedin.kmf.services.ProduceService", -# "topic": "kafka-monitor-topic", +# "topic": "xinfra-monitor-topic", # "zookeeper.connect": "localhost:2181", # "bootstrap.servers": "localhost:9092", # "consume.latency.sla.ms": "20000", @@ -115,7 +115,7 @@ # Example consume-service to consume messages # "consume-service": { # "class.name": "com.linkedin.kmf.services.ConsumeService", -# "topic": "kafka-monitor-topic", +# "topic": "xinfra-monitor-topic", # "zookeeper.connect": "localhost:2181", # "bootstrap.servers": "localhost:9092", # "consume.latency.sla.ms": "20000", @@ -143,7 +143,7 @@ # "report.interval.sec": 3, # "zookeeper.connect": "localhost:2181", # "bootstrap.servers": "localhost:9092", -# "topic": "kafka-monitor-topic-metrics", +# "topic": "xinfra-monitor-topic-metrics", # "report.kafka.topic.replication.factor": 1, # "report.metrics.list": [ # "kmf.services:type=produce-service,name=*:produce-availability-avg", diff --git a/docker/Dockerfile b/docker/Dockerfile index 8c6cbbd7..dcd49e6a 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -19,7 +19,7 @@ WORKDIR /opt/kafka-monitor ADD build/ build/ ADD bin/kafka-monitor-start.sh bin/kafka-monitor-start.sh ADD bin/kmf-run-class.sh bin/kmf-run-class.sh -ADD config/kafka-monitor.properties config/kafka-monitor.properties +ADD config/xinfra-monitor.properties config/xinfra-monitor.properties ADD config/log4j2.properties config/log4j2.properties ADD docker/kafka-monitor-docker-entry.sh kafka-monitor-docker-entry.sh ADD webapp/ webapp/ diff --git a/docker/kafka-monitor-docker-entry.sh b/docker/kafka-monitor-docker-entry.sh index 946b6e66..97554bb0 100755 --- a/docker/kafka-monitor-docker-entry.sh +++ b/docker/kafka-monitor-docker-entry.sh @@ -22,6 +22,6 @@ trap 'pkill java; exit 143' SIGTERM # wait for DNS services to be available sleep 10 -bin/kafka-monitor-start.sh config/kafka-monitor.properties & +bin/xinfra-monitor-start.sh config/xinfra-monitor.properties & wait $! \ No newline at end of file diff --git a/src/main/java/com/linkedin/kmf/XinfraMonitor.java b/src/main/java/com/linkedin/kmf/XinfraMonitor.java index 8a3cad14..df5e0eb9 100644 --- a/src/main/java/com/linkedin/kmf/XinfraMonitor.java +++ b/src/main/java/com/linkedin/kmf/XinfraMonitor.java @@ -168,7 +168,7 @@ public void awaitShutdown() { @SuppressWarnings("rawtypes") public static void main(String[] args) throws Exception { if (args.length <= 0) { - LOG.info("USAGE: java [options] " + XinfraMonitor.class.getName() + " config/kafka-monitor.properties"); + LOG.info("USAGE: java [options] " + XinfraMonitor.class.getName() + " config/xinfra-monitor.properties"); return; } diff --git a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java index 0a431ced..146f7603 100644 --- a/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java +++ b/src/main/java/com/linkedin/kmf/services/configs/StatsdMetricsReporterServiceConfig.java @@ -8,7 +8,7 @@ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * * - * In order to enable the StatsD metrics export, add the following section to kafka-monitor.properties file + * In order to enable the StatsD metrics export, add the following section to xinfra-monitor.properties file * */ From 0a385dc9cd3f094755b4f0b177d0b91ad76c3cac Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Wed, 3 Jun 2020 17:07:04 -0700 Subject: [PATCH 124/192] update readme (#262) Signed-off-by: Andrew Choi --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index ee7a5975..c8e34852 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ Xinfra Monitor supports Apache Kafka 0.8 to 2.0:
    1. We advise advanced users to run Xinfra Monitor with -./bin/kafka-monitor-start.sh config/xinfra-monitor.properties. The default +./bin/xinfra-monitor-start.sh config/xinfra-monitor.properties. The default xinfra-monitor.properties in the repo provides an simple example of how to monitor a single cluster. You probably need to change the value of zookeeper.connect and bootstrap.servers to point to your cluster. @@ -92,7 +92,7 @@ $ ./gradlew jar ### Start XinfraMonitor to run tests/services specified in the config file ``` -$ ./bin/kafka-monitor-start.sh config/xinfra-monitor.properties +$ ./bin/xinfra-monitor-start.sh config/xinfra-monitor.properties ``` ### Run Xinfra Monitor with arbitrary producer/consumer configuration (e.g. SASL enabled client) @@ -101,7 +101,7 @@ Edit `config/xinfra-monitor.properties` to specify custom configurations for pro consumer as well. The documentation for producer and consumer in the key/value maps can be found in the Apache Kafka wiki. ``` -$ ./bin/kafka-monitor-start.sh config/xinfra-monitor.properties +$ ./bin/xinfra-monitor-start.sh config/xinfra-monitor.properties ``` ### Run SingleClusterMonitor app to monitor kafka cluster @@ -124,7 +124,7 @@ from the destination cluster. See config/multi-cluster-monitor.properties for the full jmx path for these metrics. ``` -$ ./bin/kafka-monitor-start.sh config/multi-cluster-monitor.properties +$ ./bin/xinfra-monitor-start.sh config/multi-cluster-monitor.properties ``` ### Get metric values (e.g. service availability, message loss rate) in real-time as time series graphs From b103f3cb32a2f5fa3fbff949572b3badd21168e6 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 4 Jun 2020 12:31:13 -0700 Subject: [PATCH 125/192] 1 - Remove deprecations in produce service for recordsProduced and produceError. The behaviour will remain unchanged and is purely to get rid of library deprecations. 2 - Render fields local instead of instance variables inside src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java. These variables inside the class don't need to be instance variables because they aren't used elsewhere. The code behaviour will remain the same. 3 - Suppression of Warnings for some IntelliJ level warnings. Makes no logic changes. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1 - Remove deprecations in produce service for recordsProduced and produceError. The behaviour will remain unchanged and is purely to get rid of library deprecations. 2 - Render fields local instead of instance variables inside src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java. These variables inside the class don't need to be instance variables because they aren't used elsewhere. The code behaviour will remain the same. 3 - Suppression of Warnings for some IntelliJ level warnings. Makes no logic changes. https://docs.oracle.com/javase/8/docs/api/java/lang/SuppressWarnings.html when you use IntelliJ IDEA to code - it will occasionally bring up warning signs, conventionally through some highlighting of words and texts. at build time these suppressions get rid of these annoying warnings. warnings will show up like this if we don’t have suppressions: image In compile time + run time, these suppressions make no difference. suppressions only exist to get rid of annoying warning signs in Intellij (or some other IDEs). Testing: local deploy Signed-off-by: Andrew Choi --- .../linkedin/kmf/services/ProduceService.java | 14 ++++++++++---- .../SignalFxMetricsReporterService.java | 19 ++++++++----------- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/ProduceService.java b/src/main/java/com/linkedin/kmf/services/ProduceService.java index 23d71326..4883af7e 100644 --- a/src/main/java/com/linkedin/kmf/services/ProduceService.java +++ b/src/main/java/com/linkedin/kmf/services/ProduceService.java @@ -54,6 +54,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SuppressWarnings("rawtypes") public class ProduceService implements Service { private static final Logger LOG = LoggerFactory.getLogger(ProduceService.class); private static final String METRIC_GROUP_NAME = "produce-service"; @@ -64,7 +65,7 @@ public class ProduceService implements Service { private final String _name; private final ProduceMetrics _sensors; private KMBaseProducer _producer; - private KMPartitioner _partitioner; + private final KMPartitioner _partitioner; private ScheduledExecutorService _produceExecutor; private final ScheduledExecutorService _handleNewPartitionsExecutor; private final int _produceDelayMs; @@ -276,8 +277,12 @@ public ProduceMetrics(final Metrics metrics, final Map tags) { double availabilitySum = 0.0; int partitionNum = _partitionNum.get(); for (int partition = 0; partition < partitionNum; partition++) { - double recordsProduced = metrics.metrics().get(metrics.metricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); - double produceError = metrics.metrics().get(metrics.metricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)).value(); + double recordsProduced = (double) metrics.metrics() + .get(metrics.metricName("records-produced-rate-partition-" + partition, METRIC_GROUP_NAME, tags)) + .metricValue(); + double produceError = (double) metrics.metrics() + .get(metrics.metricName("produce-error-rate-partition-" + partition, METRIC_GROUP_NAME, tags)) + .metricValue(); // If there is no error, error rate sensor may expire and the value may be NaN. Treat NaN as 0 for error rate. if (Double.isNaN(produceError) || Double.isInfinite(produceError)) { produceError = 0; @@ -402,6 +407,7 @@ public void run() { } } + @SuppressWarnings("NullableProblems") private class ProduceServiceThreadFactory implements ThreadFactory { private final AtomicInteger _threadId = new AtomicInteger(); @@ -411,7 +417,7 @@ public Thread newThread(Runnable r) { } private class HandleNewPartitionsThreadFactory implements ThreadFactory { - public Thread newThread(Runnable r) { + public Thread newThread(@SuppressWarnings("NullableProblems") Runnable r) { return new Thread(r, _name + "-produce-service-new-partition-handler"); } } diff --git a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java index ee64f22a..8dcb3535 100644 --- a/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java +++ b/src/main/java/com/linkedin/kmf/services/SignalFxMetricsReporterService.java @@ -31,11 +31,9 @@ public class SignalFxMetricsReporterService implements Service { private final ScheduledExecutorService _executor; private final MetricRegistry _metricRegistry; private final SignalFxReporter _signalfxReporter; - private final String _signalfxUrl; - private final String _signalfxToken; - private MetricMetadata _metricMetadata; - private Map _metricMap; + private final MetricMetadata _metricMetadata; + private final Map _metricMap; private Map _dimensionsMap; public SignalFxMetricsReporterService(Map props, String name) throws Exception { @@ -44,10 +42,10 @@ public SignalFxMetricsReporterService(Map props, String name) th _name = name; _metricNames = config.getList(SignalFxMetricsReporterServiceConfig.REPORT_METRICS_CONFIG); _reportIntervalSec = config.getInt(SignalFxMetricsReporterServiceConfig.REPORT_INTERVAL_SEC_CONFIG); - _signalfxUrl = config.getString(SignalFxMetricsReporterServiceConfig.REPORT_SIGNALFX_URL); - _signalfxToken = config.getString(SignalFxMetricsReporterServiceConfig.SIGNALFX_TOKEN); + String signalfxUrl = config.getString(SignalFxMetricsReporterServiceConfig.REPORT_SIGNALFX_URL); + String signalfxToken = config.getString(SignalFxMetricsReporterServiceConfig.SIGNALFX_TOKEN); - if (StringUtils.isEmpty(_signalfxToken)) { + if (StringUtils.isEmpty(signalfxToken)) { throw new IllegalArgumentException("SignalFx token is not configured"); } @@ -60,11 +58,10 @@ public SignalFxMetricsReporterService(Map props, String name) th } SignalFxReporter.Builder sfxReportBuilder = new SignalFxReporter.Builder( - _metricRegistry, - _signalfxToken + _metricRegistry, signalfxToken ); - if (!StringUtils.isEmpty(_signalfxUrl)) { - sfxReportBuilder.setEndpoint(getSignalFxEndpoint(_signalfxUrl)); + if (!StringUtils.isEmpty(signalfxUrl)) { + sfxReportBuilder.setEndpoint(getSignalFxEndpoint(signalfxUrl)); } _signalfxReporter = sfxReportBuilder.build(); From 5a29f113084e8765c6d5b733c8a2ec35d7f53fb2 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Thu, 4 Jun 2020 14:17:22 -0700 Subject: [PATCH 126/192] Update Providing number of assignments by which the number of partitions was increased (#263) The partition count you are increasing the partitions by should equal the actual size of the new partition assignments. This PR achieves this. For instance: old partitions count = 5 new partitions count = 11 increase in partitions = 6 the size of the new partition assignments ought to be be 6 also. Note: Random assignment is adequate here because the periodic runnable for maybeReassignPartitions runs the reassignment operation on an interval. Signed-off-by: Andrew Choi --- .../MultiClusterTopicManagementService.java | 24 +++++++++++++++---- ...ultiClusterTopicManagementServiceTest.java | 23 ++++++++++-------- 2 files changed, 32 insertions(+), 15 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index d2eb667d..947cd9cc 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -338,14 +339,13 @@ static List> newPartitionAssignments(int minPartitionNum, int part int partitionDifference = minPartitionNum - partitionNum; // leader assignments - - for (BrokerMetadata brokerMetadata : brokers) { + while (newPartitionAssignments.size() != partitionDifference) { List replicas = new ArrayList<>(); // leader replica/broker - - replicas.add(brokerMetadata.id()); + int brokerMetadata = randomBroker(brokers).id(); + replicas.add(brokerMetadata); + newPartitionAssignments.add(replicas); - if (newPartitionAssignments.size() == partitionDifference) { - break; - } } // follower assignments - @@ -364,6 +364,20 @@ static List> newPartitionAssignments(int minPartitionNum, int part return newPartitionAssignments; } + private static BrokerMetadata randomBroker(Set brokers) { + int brokerSetSize = brokers.size(); + // In practicality, the Random object should be rather more shared than this. + int random = new Random().nextInt(brokerSetSize); + int index = 0; + for (BrokerMetadata brokerMetadata : brokers) { + if (index == random) + return brokerMetadata; + index++; + } + + throw new IllegalStateException("Couldn't find random broker."); + } + /** * Exposed package-private access for testing. Get the total number of partitions for a Kafka topic. * @return total number of topic partitions diff --git a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java index e97bb407..69c04118 100644 --- a/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java +++ b/src/test/java/com/linkedin/kmf/services/MultiClusterTopicManagementServiceTest.java @@ -60,9 +60,11 @@ private void startTest() { nodeSet.add(new Node(3, "host-3", 2134)); nodeSet.add(new Node(4, "host-4", 2135)); nodeSet.add(new Node(5, "host-5", 2136)); - nodeSet.add(new Node(6, "host-5", 2136)); - nodeSet.add(new Node(7, "host-5", 2136)); - nodeSet.add(new Node(8, "host-5", 2136)); + nodeSet.add(new Node(6, "host-5", 2137)); + nodeSet.add(new Node(7, "host-5", 2138)); + nodeSet.add(new Node(8, "host-5", 2139)); + nodeSet.add(new Node(9, "host-5", 2140)); + nodeSet.add(new Node(10, "host-5", 2141)); _topicManagementHelper = Mockito.mock(MultiClusterTopicManagementService.TopicManagementHelper.class); _topicManagementHelper._topic = SERVICE_TEST_TOPIC; @@ -82,17 +84,18 @@ protected void maybeAddPartitionsTest() { for (Node broker : nodeSet) { brokerMetadataSet.add(new BrokerMetadata(broker.id(), Option.apply(broker.rack()))); } + + int minPartitionNum = 14; + int partitionNum = 5; + int rf = 4; + List> newPartitionAssignments = - MultiClusterTopicManagementService.TopicManagementHelper.newPartitionAssignments(11, 5, brokerMetadataSet, 4); + MultiClusterTopicManagementService.TopicManagementHelper.newPartitionAssignments(minPartitionNum, partitionNum, brokerMetadataSet, rf); Assert.assertNotNull(newPartitionAssignments); System.out.println(newPartitionAssignments); - Assert.assertEquals(newPartitionAssignments.get(0).get(0).intValue(), 1); - Assert.assertEquals(newPartitionAssignments.get(1).get(0).intValue(), 2); - Assert.assertEquals(newPartitionAssignments.get(2).get(0).intValue(), 3); - Assert.assertEquals(newPartitionAssignments.get(3).get(0).intValue(), 4); - Assert.assertEquals(newPartitionAssignments.get(4).get(0).intValue(), 5); - Assert.assertEquals(newPartitionAssignments.get(5).get(0).intValue(), 6); + Assert.assertEquals(newPartitionAssignments.size(), minPartitionNum - partitionNum); + Assert.assertEquals(newPartitionAssignments.get(0).size(), rf); } @Test From 1a3666f636ce71bc4faf2a1123cca2da366a0804 Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 5 Jun 2020 12:19:55 -0700 Subject: [PATCH 127/192] Conversion of Set to List for BrokerMetadata Collection #264 Set is unordered and doesn't allow for duplicates. List is ordered and allows for duplicates. Converting Set -> List for O(1) runtime complexity in lieu of O(n) caused by the usage of List. Set -> List should be safe in this case since the Set contains non-duplicated values. TopicFactory interface declares the following member functions. Changing this method to return List could break the backward compatibility Set getBlackListedBrokers(String zkUrl); Signed-off-by: Andrew Choi --- .../MultiClusterTopicManagementService.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java index 947cd9cc..7fb95c03 100644 --- a/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java +++ b/src/main/java/com/linkedin/kmf/services/MultiClusterTopicManagementService.java @@ -335,7 +335,7 @@ static List> newPartitionAssignments(int minPartitionNum, int part // partition 3's preferred leader will be broker 1, // partition 4's preferred leader will be broker 2 and // partition 5's preferred leader will be broker 3. - List> newPartitionAssignments = new ArrayList<>(new ArrayList<>()); + List> newPartitionAssignments = new ArrayList<>(); int partitionDifference = minPartitionNum - partitionNum; // leader assignments - @@ -365,17 +365,25 @@ static List> newPartitionAssignments(int minPartitionNum, int part } private static BrokerMetadata randomBroker(Set brokers) { + + if (brokers == null || brokers.size() == 0) { + throw new IllegalArgumentException("brokers object is either null or empty."); + } + + // Using Set enforces the usage of loop which is O(n). + // As the list of brokers does not change in newPartitionAssignments, + // the acceptance of a List argument instead of a Set will be faster which is (O(1)) + List brokerMetadataList = new ArrayList<>(); + + // convert to a list so there's no need to create a index and iterate through this set + brokerMetadataList.addAll(brokers); + int brokerSetSize = brokers.size(); + // In practicality, the Random object should be rather more shared than this. int random = new Random().nextInt(brokerSetSize); - int index = 0; - for (BrokerMetadata brokerMetadata : brokers) { - if (index == random) - return brokerMetadata; - index++; - } - throw new IllegalStateException("Couldn't find random broker."); + return brokerMetadataList.get(random); } /** From 22e66b9aae54c87b3f8ee444cb721a14fd0b6b4a Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Fri, 5 Jun 2020 18:26:38 -0700 Subject: [PATCH 128/192] Travis CI clean up instead of the deprecated skip_cleanup under .travis YML file #265 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Travis CI has deprecated the use of the key skip_cleanup. Thus, have Travis CI use clean up instead of the deprecated skip_cleanup under .travis YML file Why? Such that the following tests don't fail. 1. continuous-integration/travis-ci/pr — The Travis CI build passed 2. continuous-integration/travis-ci/push — The Travis CI build passed Signed-off-by: Andrew Choi --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index d866cfeb..511503f0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -28,7 +28,7 @@ cache: deploy: provider: script script: bash -v scripts/publishToBintray.sh - skip_cleanup: true + cleanup: true on: all_branches: true tags: true From 23d2d5618cac91834f4f4cfbdc093abafff83d3f Mon Sep 17 00:00:00 2001 From: Andrew Choi Date: Mon, 8 Jun 2020 12:56:47 -0700 Subject: [PATCH 129/192] Minor, Update web application index HTML file -- Updated license year to 2020 Minor, Update web application index HTML file -- Updated license year to 2020 --- webapp/index.html | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/webapp/index.html b/webapp/index.html index 5c2a86d4..03c22fb8 100644 --- a/webapp/index.html +++ b/webapp/index.html @@ -1,5 +1,5 @@