Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2480#discussion_r158958894
  
    --- Diff: 
external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
 ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.metrics;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.storm.kafka.spout.internal.OffsetManager;
    +import org.apache.storm.metric.api.IMetric;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class KafkaOffsetMetric implements IMetric {
    +
    +    private static final Logger LOG = 
LoggerFactory.getLogger(KafkaOffsetMetric.class);
    +    private Map<TopicPartition, OffsetManager> offsetManagers;
    +    private KafkaConsumer kafkaConsumer;
    +
    +    public KafkaOffsetMetric(Map<TopicPartition, OffsetManager> 
offsetManagers) {
    +        this.offsetManagers = offsetManagers;
    +    }
    +
    +    @Override
    +    public Object getValueAndReset() {
    +        try {
    +            HashMap<String, Long> ret = new HashMap<>();
    +            if (offsetManagers != null && !offsetManagers.isEmpty() && 
kafkaConsumer != null) {
    +                Map<String,TopicMetrics> topicMetricsMap = new HashMap<>();
    +                Set<TopicPartition> topicPartitions = 
offsetManagers.keySet();
    +
    +                Map<TopicPartition, Long> beginningOffsets= 
kafkaConsumer.beginningOffsets(topicPartitions);
    +                Map<TopicPartition, Long> endOffsets= 
kafkaConsumer.endOffsets(topicPartitions);
    +
    +                for (Map.Entry<TopicPartition, OffsetManager> entry : 
offsetManagers.entrySet()) {
    +                    TopicPartition topicPartition = entry.getKey();
    +                    OffsetManager offsetManager = entry.getValue();
    +
    +                    long latestTimeOffset = endOffsets.get(topicPartition);
    +                    long earliestTimeOffset = 
beginningOffsets.get(topicPartition);
    +
    +                    long latestEmittedOffset = 
offsetManager.getLatestEmittedOffset();
    +                    long latestCompletedOffset = 
offsetManager.getCommittedOffset();
    +                    long spoutLag = latestTimeOffset - 
latestCompletedOffset;
    +
    +                    String metricPath = topicPartition.topic()  + 
"/partition_" + topicPartition.partition();
    +                    ret.put(metricPath + "/" + "spoutLag", spoutLag);
    +                    ret.put(metricPath + "/" + "earliestTimeOffset", 
earliestTimeOffset);
    +                    ret.put(metricPath + "/" + "latestTimeOffset", 
latestTimeOffset);
    +                    ret.put(metricPath + "/" + "latestEmittedOffset", 
latestEmittedOffset);
    +                    ret.put(metricPath + "/" + "latestCompletedOffset", 
latestCompletedOffset);
    +
    +                    TopicMetrics topicMetrics = 
topicMetricsMap.get(topicPartition.topic());
    +                    if(topicMetrics == null) {
    +                        topicMetrics = new TopicMetrics();
    +                        topicMetricsMap.put(topicPartition.topic(), 
topicMetrics);
    +                    }
    +
    +                    topicMetrics.totalSpoutLag += spoutLag;
    +                    topicMetrics.totalEarliestTimeOffset += 
earliestTimeOffset;
    +                    topicMetrics.totalLatestTimeOffset += latestTimeOffset;
    +                    topicMetrics.totalLatestEmittedOffset += 
latestEmittedOffset;
    +                    topicMetrics.totalLatestCompletedOffset += 
latestCompletedOffset;
    +                }
    +
    +                for(Map.Entry<String, TopicMetrics> e : 
topicMetricsMap.entrySet()) {
    +                    String topic = e.getKey();
    +                    TopicMetrics topicMetrics = e.getValue();
    +                    ret.put(topic + "/" + "totalSpoutLag", 
topicMetrics.totalSpoutLag);
    +                    ret.put(topic + "/" + "totalEarliestTimeOffset", 
topicMetrics.totalEarliestTimeOffset);
    +                    ret.put(topic + "/" + "totalLatestTimeOffset", 
topicMetrics.totalLatestTimeOffset);
    +                    ret.put(topic + "/" + "totalLatestEmittedOffset", 
topicMetrics.totalLatestEmittedOffset);
    +                    ret.put(topic + "/" + "totalLatestCompletedOffset", 
topicMetrics.totalLatestCompletedOffset);
    +                }
    +                LOG.debug("Metrics Tick: value : {}", ret);
    +                return ret;
    +            } else {
    --- End diff --
    
    Nit: Invert the if-else here to reduce indentation. It also allows you to 
avoid the "catch-all" null return in the bottom.


---

Reply via email to