-
Notifications
You must be signed in to change notification settings - Fork 4.1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[STORM-3782] Refactor KafkaOffsetMetric to use V2 metrics (#3404)
* [STORM-3782] Refactor KafkaOffsetMetric to use V2 metrics
- Loading branch information
Showing
8 changed files
with
445 additions
and
230 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
151 changes: 0 additions & 151 deletions
151
...rm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
This file was deleted.
Oops, something went wrong.
85 changes: 85 additions & 0 deletions
85
...-client/src/main/java/org/apache/storm/kafka/spout/metrics2/KafkaOffsetMetricManager.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.storm.kafka.spout.metrics2; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.function.Supplier; | ||
|
||
import org.apache.kafka.clients.consumer.Consumer; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.storm.kafka.spout.internal.OffsetManager; | ||
import org.apache.storm.task.TopologyContext; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* This class is used to manage both the partition and topic level offset metrics. | ||
*/ | ||
public class KafkaOffsetMetricManager<K, V> { | ||
private static final Logger LOG = LoggerFactory.getLogger(KafkaOffsetMetricManager.class); | ||
private final Supplier<Map<TopicPartition, OffsetManager>> offsetManagerSupplier; | ||
private final Supplier<Consumer<K, V>> consumerSupplier; | ||
private TopologyContext topologyContext; | ||
|
||
private Map<String, KafkaOffsetTopicMetrics> topicMetricsMap; | ||
private Map<TopicPartition, KafkaOffsetPartitionMetrics> topicPartitionMetricsMap; | ||
|
||
public KafkaOffsetMetricManager(Supplier<Map<TopicPartition, OffsetManager>> offsetManagerSupplier, | ||
Supplier<Consumer<K, V>> consumerSupplier, | ||
TopologyContext topologyContext) { | ||
this.offsetManagerSupplier = offsetManagerSupplier; | ||
this.consumerSupplier = consumerSupplier; | ||
this.topologyContext = topologyContext; | ||
|
||
this.topicMetricsMap = new HashMap<>(); | ||
this.topicPartitionMetricsMap = new HashMap<>(); | ||
LOG.info("Running KafkaOffsetMetricManager"); | ||
} | ||
|
||
public void registerMetricsForNewTopicPartitions(Set<TopicPartition> newAssignment) { | ||
for (TopicPartition topicPartition : newAssignment) { | ||
if (!topicPartitionMetricsMap.containsKey(topicPartition)) { | ||
LOG.info("Registering metric for topicPartition: {}", topicPartition); | ||
// create topic level metrics for given topic if absent | ||
String topic = topicPartition.topic(); | ||
KafkaOffsetTopicMetrics topicMetrics = topicMetricsMap.get(topic); | ||
if (topicMetrics == null) { | ||
topicMetrics = new KafkaOffsetTopicMetrics(topic); | ||
topicMetricsMap.put(topic, topicMetrics); | ||
topologyContext.registerMetricSet("kafkaOffset", topicMetrics); | ||
} | ||
|
||
KafkaOffsetPartitionMetrics topicPartitionMetricSet | ||
= new KafkaOffsetPartitionMetrics<>(offsetManagerSupplier, consumerSupplier, topicPartition, topicMetrics); | ||
topicPartitionMetricsMap.put(topicPartition, topicPartitionMetricSet); | ||
topologyContext.registerMetricSet("kafkaOffset", topicPartitionMetricSet); | ||
} | ||
} | ||
} | ||
|
||
public Map<TopicPartition, KafkaOffsetPartitionMetrics> getTopicPartitionMetricsMap() { | ||
return topicPartitionMetricsMap; | ||
} | ||
|
||
public Map<String, KafkaOffsetTopicMetrics> getTopicMetricsMap() { | ||
return topicMetricsMap; | ||
} | ||
} |
Oops, something went wrong.