hlteoh37 commented on code in PR #143:
URL: 
https://github.com/apache/flink-connector-aws/pull/143#discussion_r1644537365


##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetrics.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.arns.Arn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A utility class for handling metrics in {@link
+ * 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader}.
+ */
+@Internal
+public class KinesisSourceShardMetrics {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisSourceShardMetrics.class);
+
+    private final Map<String, Long> millisBehindLatestMap = new HashMap<>();
+    private final Map<String, MetricGroup> shardMetricGroupMap = new 
HashMap<>();
+    private final MetricGroup sourceReaderMetricGroup;
+
+    public KinesisSourceShardMetrics(final MetricGroup metricGroup) {
+        this.sourceReaderMetricGroup = metricGroup;
+    }
+
+    public void registerShardMetricGroup(KinesisShardSplit shard) {
+        LOG.debug("Registering the metric group for shard {}", 
shard.getShardId());
+        Arn shardArn = Arn.fromString(shard.getStreamArn());

Review Comment:
   is this `shardArn` or `streamArn`?



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetrics.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.arns.Arn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A utility class for handling metrics in {@link
+ * 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader}.
+ */
+@Internal
+public class KinesisSourceShardMetrics {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisSourceShardMetrics.class);
+
+    private final Map<String, Long> millisBehindLatestMap = new HashMap<>();
+    private final Map<String, MetricGroup> shardMetricGroupMap = new 
HashMap<>();
+    private final MetricGroup sourceReaderMetricGroup;
+
+    public KinesisSourceShardMetrics(final MetricGroup metricGroup) {
+        this.sourceReaderMetricGroup = metricGroup;
+    }
+
+    public void registerShardMetricGroup(KinesisShardSplit shard) {
+        LOG.debug("Registering the metric group for shard {}", 
shard.getShardId());
+        Arn shardArn = Arn.fromString(shard.getStreamArn());
+
+        millisBehindLatestMap.put(shard.getShardId(), -1L);
+        final MetricGroup shardGroup =
+                this.sourceReaderMetricGroup
+                        .addGroup(
+                                MetricConstants.ACCOUNT_ID_METRIC_GROUP, 
shardArn.accountId().get())
+                        .addGroup(MetricConstants.REGION_METRIC_GROUP, 
shardArn.region().get())
+                        .addGroup(
+                                MetricConstants.STREAM_METRIC_GROUP, 
shardArn.resource().resource())
+                        .addGroup(MetricConstants.SHARD_METRIC_GROUP, 
shard.getShardId());
+
+        shardGroup.gauge(
+                MetricConstants.MILLIS_BEHIND_LATEST_GAUGE,
+                () -> millisBehindLatestMap.get(shard.getShardId()));
+        shardMetricGroupMap.put(shard.getShardId(), shardGroup);
+
+        LOG.debug(
+                "Registered metric identifier {}",
+                
shardGroup.getMetricIdentifier(MetricConstants.MILLIS_BEHIND_LATEST_GAUGE));
+    }
+
+    public void updateShardMillisBehindLatest(
+            KinesisShardSplitState shardState, Long millisBehindLatest) {
+        LOG.debug(
+                "Updating millisBehindLatest metric for shard {} to {}",
+                shardState.getShardId(),
+                millisBehindLatest);
+        millisBehindLatestMap.put(shardState.getShardId(), millisBehindLatest);
+    }
+
+    public void unregisterShardMetricGroup(KinesisShardSplitState shardState) {
+        if (shardMetricGroupMap.containsKey(shardState.getShardId())) {
+            MetricGroup shardMetricGroup = 
shardMetricGroupMap.get(shardState.getShardId());
+
+            if (shardMetricGroup instanceof AbstractMetricGroup) {
+                ((AbstractMetricGroup<?>) shardMetricGroup).close();
+            }
+
+            shardMetricGroupMap.remove(shardState.getShardId());
+        } else {
+            LOG.warn(
+                    "Shard metric group unregister failed. Metric group for {} 
does not exist.",
+                    shardState.getShardId());
+        }
+    }
+
+    @VisibleForTesting

Review Comment:
   This is not in line with Flink's testing principles. Is there a reason we 
cannot set this up using a `TestMetricGroup` instead?



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetrics.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.arns.Arn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A utility class for handling metrics in {@link
+ * 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader}.
+ */
+@Internal
+public class KinesisSourceShardMetrics {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisSourceShardMetrics.class);
+
+    private final Map<String, Long> millisBehindLatestMap = new HashMap<>();
+    private final Map<String, MetricGroup> shardMetricGroupMap = new 
HashMap<>();
+    private final MetricGroup sourceReaderMetricGroup;
+
+    public KinesisSourceShardMetrics(final MetricGroup metricGroup) {
+        this.sourceReaderMetricGroup = metricGroup;
+    }
+
+    public void registerShardMetricGroup(KinesisShardSplit shard) {

Review Comment:
   Do we want to do a check to validate that the shard is not already 
registered? 
   



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetrics.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.arns.Arn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A utility class for handling metrics in {@link
+ * 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader}.
+ */
+@Internal
+public class KinesisSourceShardMetrics {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisSourceShardMetrics.class);
+
+    private final Map<String, Long> millisBehindLatestMap = new HashMap<>();
+    private final Map<String, MetricGroup> shardMetricGroupMap = new 
HashMap<>();
+    private final MetricGroup sourceReaderMetricGroup;
+
+    public KinesisSourceShardMetrics(final MetricGroup metricGroup) {
+        this.sourceReaderMetricGroup = metricGroup;
+    }
+
+    public void registerShardMetricGroup(KinesisShardSplit shard) {
+        LOG.debug("Registering the metric group for shard {}", 
shard.getShardId());
+        Arn shardArn = Arn.fromString(shard.getStreamArn());
+
+        millisBehindLatestMap.put(shard.getShardId(), -1L);
+        final MetricGroup shardGroup =
+                this.sourceReaderMetricGroup
+                        .addGroup(
+                                MetricConstants.ACCOUNT_ID_METRIC_GROUP, 
shardArn.accountId().get())
+                        .addGroup(MetricConstants.REGION_METRIC_GROUP, 
shardArn.region().get())
+                        .addGroup(
+                                MetricConstants.STREAM_METRIC_GROUP, 
shardArn.resource().resource())
+                        .addGroup(MetricConstants.SHARD_METRIC_GROUP, 
shard.getShardId());
+
+        shardGroup.gauge(
+                MetricConstants.MILLIS_BEHIND_LATEST_GAUGE,
+                () -> millisBehindLatestMap.get(shard.getShardId()));
+        shardMetricGroupMap.put(shard.getShardId(), shardGroup);
+
+        LOG.debug(
+                "Registered metric identifier {}",
+                
shardGroup.getMetricIdentifier(MetricConstants.MILLIS_BEHIND_LATEST_GAUGE));
+    }
+
+    public void updateShardMillisBehindLatest(
+            KinesisShardSplitState shardState, Long millisBehindLatest) {
+        LOG.debug(
+                "Updating millisBehindLatest metric for shard {} to {}",
+                shardState.getShardId(),
+                millisBehindLatest);
+        millisBehindLatestMap.put(shardState.getShardId(), millisBehindLatest);
+    }
+
+    public void unregisterShardMetricGroup(KinesisShardSplitState shardState) {

Review Comment:
   nit: `deregister`



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetrics.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.arns.Arn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A utility class for handling metrics in {@link
+ * 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader}.
+ */
+@Internal
+public class KinesisSourceShardMetrics {
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisSourceShardMetrics.class);
+
+    private final Map<String, Long> millisBehindLatestMap = new HashMap<>();
+    private final Map<String, MetricGroup> shardMetricGroupMap = new 
HashMap<>();
+    private final MetricGroup sourceReaderMetricGroup;
+
+    public KinesisSourceShardMetrics(final MetricGroup metricGroup) {
+        this.sourceReaderMetricGroup = metricGroup;
+    }
+
+    public void registerShardMetricGroup(KinesisShardSplit shard) {
+        LOG.debug("Registering the metric group for shard {}", 
shard.getShardId());
+        Arn shardArn = Arn.fromString(shard.getStreamArn());
+
+        millisBehindLatestMap.put(shard.getShardId(), -1L);
+        final MetricGroup shardGroup =
+                this.sourceReaderMetricGroup
+                        .addGroup(
+                                MetricConstants.ACCOUNT_ID_METRIC_GROUP, 
shardArn.accountId().get())
+                        .addGroup(MetricConstants.REGION_METRIC_GROUP, 
shardArn.region().get())
+                        .addGroup(
+                                MetricConstants.STREAM_METRIC_GROUP, 
shardArn.resource().resource())
+                        .addGroup(MetricConstants.SHARD_METRIC_GROUP, 
shard.getShardId());
+
+        shardGroup.gauge(
+                MetricConstants.MILLIS_BEHIND_LATEST_GAUGE,
+                () -> millisBehindLatestMap.get(shard.getShardId()));
+        shardMetricGroupMap.put(shard.getShardId(), shardGroup);
+
+        LOG.debug(
+                "Registered metric identifier {}",
+                
shardGroup.getMetricIdentifier(MetricConstants.MILLIS_BEHIND_LATEST_GAUGE));
+    }
+
+    public void updateShardMillisBehindLatest(
+            KinesisShardSplitState shardState, Long millisBehindLatest) {

Review Comment:
   We can consider reducing the argument here to minimal necessary (just 
shardId)



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetricsTest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.connector.kinesis.source.util.TestUtil;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.arns.Arn;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static 
org.assertj.core.api.AssertionsForClassTypes.assertThatNoException;
+
+class KinesisSourceShardMetricsTest {
+    private static final KinesisShardSplit TEST_SPLIT =
+            TestUtil.getTestSplit(TestUtil.STREAM_ARN, 
TestUtil.generateShardId(1));
+    private static final KinesisShardSplitState TEST_SPLIT_STATE =
+            TestUtil.getTestSplitState(TEST_SPLIT);
+
+    private MetricListener metricListener;
+    private KinesisSourceShardMetrics kinesisSourceShardMetrics;
+
+    @BeforeEach
+    public void init() {
+        metricListener = new MetricListener();
+
+        kinesisSourceShardMetrics =
+                new KinesisSourceShardMetrics(
+                        metricListener
+                                .getMetricGroup()
+                                
.addGroup(MetricConstants.KINESIS_STREAM_SOURCE_METRIC_GROUP));
+
+        kinesisSourceShardMetrics.registerShardMetricGroup(TEST_SPLIT);
+    }
+
+    @Test
+    void testRegisterShardMetricGroup() {
+        TestUtil.assertMillisBehindLatest(TEST_SPLIT, -1L, metricListener);
+    }
+
+    @Test
+    void testKinesisMetricIdentifier() {
+        Arn streamArn = Arn.fromString(TEST_SPLIT.getStreamArn());
+
+        String expectedIdentifier =
+                String.format(
+                        "%s.%s.%s.%s.%s.%s.%s.%s.%s",

Review Comment:
   nit: we might be better off using "." join here... This is not very 
readable...



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/source/metrics/KinesisSourceShardMetricsTest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.flink.connector.kinesis.source.metrics;
+
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.connector.kinesis.source.util.TestUtil;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.arns.Arn;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static 
org.assertj.core.api.AssertionsForClassTypes.assertThatNoException;
+
+class KinesisSourceShardMetricsTest {
+    private static final KinesisShardSplit TEST_SPLIT =
+            TestUtil.getTestSplit(TestUtil.STREAM_ARN, 
TestUtil.generateShardId(1));
+    private static final KinesisShardSplitState TEST_SPLIT_STATE =
+            TestUtil.getTestSplitState(TEST_SPLIT);
+
+    private MetricListener metricListener;
+    private KinesisSourceShardMetrics kinesisSourceShardMetrics;
+
+    @BeforeEach
+    public void init() {
+        metricListener = new MetricListener();
+
+        kinesisSourceShardMetrics =
+                new KinesisSourceShardMetrics(
+                        metricListener
+                                .getMetricGroup()
+                                
.addGroup(MetricConstants.KINESIS_STREAM_SOURCE_METRIC_GROUP));
+
+        kinesisSourceShardMetrics.registerShardMetricGroup(TEST_SPLIT);
+    }
+
+    @Test
+    void testRegisterShardMetricGroup() {
+        TestUtil.assertMillisBehindLatest(TEST_SPLIT, -1L, metricListener);
+    }
+
+    @Test
+    void testKinesisMetricIdentifier() {
+        Arn streamArn = Arn.fromString(TEST_SPLIT.getStreamArn());
+
+        String expectedIdentifier =
+                String.format(
+                        "%s.%s.%s.%s.%s.%s.%s.%s.%s",
+                        MetricConstants.KINESIS_STREAM_SOURCE_METRIC_GROUP,
+                        MetricConstants.ACCOUNT_ID_METRIC_GROUP,
+                        streamArn.accountId().get(),
+                        MetricConstants.REGION_METRIC_GROUP,
+                        streamArn.region().get(),
+                        MetricConstants.STREAM_METRIC_GROUP,
+                        streamArn.resource().resource(),
+                        MetricConstants.SHARD_METRIC_GROUP,
+                        TEST_SPLIT.getShardId());
+
+        String identifier =
+                kinesisSourceShardMetrics
+                        .getShardMetricGroupMap()
+                        .get(TEST_SPLIT.getShardId())
+                        
.getMetricIdentifier(MetricConstants.MILLIS_BEHIND_LATEST_GAUGE);
+
+        assertThat(identifier).contains(expectedIdentifier);
+    }
+
+    @Test
+    void testUpdateShardMillisBehindLatest() {
+        
kinesisSourceShardMetrics.updateShardMillisBehindLatest(TEST_SPLIT_STATE, 100L);
+        TestUtil.assertMillisBehindLatest(TEST_SPLIT, 100L, metricListener);

Review Comment:
   nice! :) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to