From 52421b9005349a4edf0382d91a3e3ec44c245c39 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 17:55:24 +0530 Subject: [PATCH 01/53] Update AbstractMetrics.java --- .../java/org/apache/pinot/common/metrics/AbstractMetrics.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index 4c879e127868..e776dd0c04f3 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -32,6 +32,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.Utils; import org.apache.pinot.spi.metrics.PinotGauge; import org.apache.pinot.spi.metrics.PinotMeter; From dcf4bd91bc5a991b3b9ecaedad24c9383d49ef5a Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:05:21 +0530 Subject: [PATCH 02/53] Update IngestionDelayTracker.java --- .../realtime/IngestionDelayTracker.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index e26390b9e4fb..907453d0b5f4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.metrics.ServerGauge; import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; @@ -98,12 +99,15 @@ private static class IngestionInfo { @Nullable volatile StreamPartitionMsgOffset _currentOffset; volatile long _firstStreamIngestionTimeMs; + @Nullable + final String _streamTopicName; IngestionInfo(long ingestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset, - long firstStreamIngestionTimeMs) { + long firstStreamIngestionTimeMs, @Nullable String streamTopicName) { _ingestionTimeMs = ingestionTimeMs; _currentOffset = currentOffset; _firstStreamIngestionTimeMs = firstStreamIngestionTimeMs; + _streamTopicName = streamTopicName; } void update(long ingestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset, @@ -394,7 +398,7 @@ void createMetrics(int partitionId) { LOGGER.info("Successfully created ingestion metrics for partition id: {}", partitionId); } - private void removeMetrics(int partitionId) { + private void removeMetrics(int partitionId, @Nullable String streamTopicName) { int streamConfigIndex = IngestionConfigUtils.getStreamConfigIndexFromPinotPartitionId(partitionId); StreamMetadataProvider streamMetadataProvider = _streamConfigIndexToStreamMetadataProvider.get(streamConfigIndex); @@ -410,6 +414,12 @@ private void removeMetrics(int partitionId) { ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS); + if (StringUtils.isNotBlank(streamTopicName)) { + _serverMetrics.removePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, + ServerGauge.REALTIME_INGESTION_DELAY_MS); + _serverMetrics.removePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, + ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); + } LOGGER.info("Successfully removed ingestion metrics for partition id: {}", partitionId); } @@ -420,12 +430,14 @@ private void removeMetrics(int partitionId) { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing the * segment name) + * @param streamTopicName stream topic name for this consumer (e.g. Kafka topic); when blank, metrics use the legacy + * name without a topic label * @param ingestionTimeMs ingestion time of the last consumed message (from {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) * @param currentOffset offset of the last consumed message (from {@link StreamMessageMetadata}) */ - public void updateMetrics(String segmentName, int partitionId, long ingestionTimeMs, + public void updateMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, long ingestionTimeMs, long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { if (!_isServerReadyToServeQueries.getAsBoolean() || _realTimeTableDataManager.isShutDown()) { // Do not update the ingestion delay metrics during server startup period @@ -444,7 +456,7 @@ public void updateMetrics(String segmentName, int partitionId, long ingestionTim return v; } if (v == null) { - return new IngestionInfo(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs); + return new IngestionInfo(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs, streamTopicName); } v.update(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs); return v; From 6ef8adacdd9638075c586485e785eddeb50f3867 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:06:45 +0530 Subject: [PATCH 03/53] Update RealtimeTableDataManager.java --- .../core/data/manager/realtime/RealtimeTableDataManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index d3849e49e1b0..5928b2f8c28f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -310,15 +310,16 @@ protected void doShutdown() { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing * the segment name) + * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion delay metric labels * @param ingestionTimeMs ingestion time of the last consumed message (from * {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) * @param currentOffset offset of the last consumed message (from {@link StreamMessageMetadata}) */ - public void updateIngestionMetrics(String segmentName, int partitionId, long ingestionTimeMs, + public void updateIngestionMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, long ingestionTimeMs, long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { - _ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, firstStreamIngestionTimeMs, + _ingestionDelayTracker.updateMetrics(segmentName, partitionId, streamTopicName, ingestionTimeMs, firstStreamIngestionTimeMs, currentOffset); } From b7e6a3ed41a7280e9fdbbad0629f203a84995ad1 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:11:28 +0530 Subject: [PATCH 04/53] Update IngestionDelayTrackerTest.java --- .../realtime/IngestionDelayTrackerTest.java | 48 +++++++++++++------ 1 file changed, 33 insertions(+), 15 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index 80d5444e3ed9..a01fd258d6e5 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -50,8 +50,11 @@ import org.testng.Assert; import org.testng.annotations.Test; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; public class IngestionDelayTrackerTest { @@ -210,7 +213,7 @@ public void testRecordIngestionDelayWithNoAging() { // Test we follow a single partition up and down for (long ingestionTimeMs = 0; ingestionTimeMs <= maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -222,7 +225,7 @@ public void testRecordIngestionDelayWithNoAging() { // Test tracking down a measure for a given partition for (long ingestionTimeMs = maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -231,12 +234,12 @@ public void testRecordIngestionDelayWithNoAging() { } // Make the current partition maximum - ingestionDelayTracker.updateMetrics(segment0, partition0, maxTestDelay, maxTestDelay, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, maxTestDelay, maxTestDelay, null); // Bring up partition1 delay up and verify values for (long ingestionTimeMs = 0; ingestionTimeMs <= 2 * maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -247,7 +250,7 @@ public void testRecordIngestionDelayWithNoAging() { // Bring down values of partition1 and verify values for (long ingestionTimeMs = 2 * maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -280,7 +283,7 @@ public void testRecordIngestionDelayWithAging() { Clock clock = Clock.fixed(now, zoneId); ingestionDelayTracker.setClock(clock); long ingestionTimeMs = clock.millis() - partition0Delay0; - ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), partition0Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), partition0Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); @@ -295,7 +298,7 @@ public void testRecordIngestionDelayWithAging() { Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); ingestionTimeMs = offsetClock.millis() - partition0Delay1; - ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), partition0Delay1); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), partition0Delay1); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); @@ -337,7 +340,7 @@ public void testStopTrackingIngestionDelay() { for (int partitionId = 0; partitionId <= maxTestDelay; partitionId++) { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - partitionId; - ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, partitionId, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partitionId), ingestionTimeMs); @@ -365,7 +368,7 @@ public void testStopTrackingIngestionDelayWithSegment() { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - 10; - ingestionDelayTracker.updateMetrics(segmentName, 0, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, 0, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(0), 10); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0), 10); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), ingestionTimeMs); @@ -376,7 +379,7 @@ public void testStopTrackingIngestionDelayWithSegment() { Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), Long.MIN_VALUE); // Should not update metrics for removed segment - ingestionDelayTracker.updateMetrics(segmentName, 0, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, 0, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertNull(ingestionDelayTracker.getPartitionIngestionDelayMs(0)); Assert.assertNull(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0)); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), Long.MIN_VALUE); @@ -397,7 +400,7 @@ public void testShutdown() { for (int partitionId = 0; partitionId <= maxTestDelay; partitionId++) { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - partitionId; - ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, partitionId, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partitionId), partitionId); } @@ -423,7 +426,7 @@ public void testRecordIngestionDelayOffset() { StreamPartitionMsgOffset msgOffset0 = new LongMsgOffset(50); StreamPartitionMsgOffset latestOffset0 = new LongMsgOffset(150); partitionMsgOffsetMap.put(partition0, latestOffset0); - ingestionDelayTracker.updateMetrics(segment0, partition0, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition0), 100); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition0), 150); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition0), 50); @@ -434,7 +437,7 @@ public void testRecordIngestionDelayOffset() { StreamPartitionMsgOffset msgOffset1 = new LongMsgOffset(50); StreamPartitionMsgOffset latestOffset1 = new LongMsgOffset(150); partitionMsgOffsetMap.put(partition1, latestOffset1); - ingestionDelayTracker.updateMetrics(segment1, partition1, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset1); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset1); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition1), 100); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition1), 150); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition1), 50); @@ -444,7 +447,7 @@ public void testRecordIngestionDelayOffset() { msgOffset0 = new LongMsgOffset(150); latestOffset0 = new LongMsgOffset(200); partitionMsgOffsetMap.put(partition0, latestOffset0); - ingestionDelayTracker.updateMetrics(segment0, partition0, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition0), 50); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition0), 200); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition0), 150); @@ -503,7 +506,7 @@ public void testIngestionDelay() { ingestionDelayTracker._partitionsHostedByThisServer.put(partition0, true); ingestionDelayTracker._partitionsHostedByThisServer.put(partition1, true); - ingestionDelayTracker.updateMetrics(segment0, partition0, System.currentTimeMillis(), System.currentTimeMillis(), + ingestionDelayTracker.updateMetrics(segment0, partition0, null, System.currentTimeMillis(), System.currentTimeMillis(), new LongMsgOffset(50)); ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); @@ -531,6 +534,21 @@ public void testIngestionDelay() { ingestionDelayTracker.shutdown(); } + @Test + public void testStreamTopicRegistersTopicTaggedGauges() { + IngestionDelayTracker tracker = createTracker(); + Instant now = Instant.now(); + tracker.setClock(Clock.fixed(now, ZoneId.systemDefault())); + String segment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); + long ts = now.toEpochMilli(); + tracker.updateIngestionMetrics(segment, 0, "test_topic", ts, ts, null, null); + verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), + eq(ServerGauge.REALTIME_INGESTION_DELAY_MS), any()); + verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), + eq(ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS), any()); + tracker.shutdown(); + } + private void verifyMetrics(Map>> partitionToMetricToValues) { Assert.assertEquals(partitionToMetricToValues.size(), 2); verifyPartition0(partitionToMetricToValues.get(0)); From 45264854aea5bd4995698ffa1e5a22258343bba7 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:49:48 +0530 Subject: [PATCH 05/53] Update PinotPrometheusMetricsTest.java --- .../common/metrics/prometheus/PinotPrometheusMetricsTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java index 4dd04ffac31d..ac7a0c415438 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java @@ -314,6 +314,10 @@ public static class ExportedLabels { TableType.REALTIME.toString(), TOPIC, KAFKA_TOPIC); public static final List PARTITION_TABLENAME_TABLETYPE = List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString()); + + public static final List PARTITIONNUM_TABLENAME_TABLETYPE_KAFKATOPIC = + List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString(), TOPIC, + KAFKA_TOPIC); public static final List TABLENAME_TABLETYPE_CONTROLLER_TASKTYPE = List.of(TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TABLETYPE_REALTIME, TASKTYPE, From 36237ef4be8b1978eb3d53964fa4fd89122f1ba1 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:58:49 +0530 Subject: [PATCH 06/53] Update ServerPrometheusMetricsTest.java --- .../prometheus/ServerPrometheusMetricsTest.java | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java index 79fb88f56ed7..31de180cc203 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java @@ -54,9 +54,12 @@ public abstract class ServerPrometheusMetricsTest extends PinotPrometheusMetrics private static final List GAUGES_ACCEPTING_PARTITION = List.of(ServerGauge.UPSERT_VALID_DOC_ID_SNAPSHOT_COUNT, ServerGauge.UPSERT_PRIMARY_KEYS_IN_SNAPSHOT_COUNT, - ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.REALTIME_INGESTION_DELAY_MS, - ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT, - ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); + ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, + ServerGauge.DEDUP_PRIMARY_KEYS_COUNT, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, + ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); + + private static final List GAUGES_ACCEPTING_PARTITION_WITH_STREAM_TOPIC = + List.of(ServerGauge.REALTIME_INGESTION_DELAY_MS, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); private static final List GAUGES_ACCEPTING_RAW_TABLE_NAME = List.of(ServerGauge.REALTIME_OFFHEAP_MEMORY_USED, ServerGauge.REALTIME_SEGMENT_NUM_PARTITIONS, @@ -122,6 +125,10 @@ public void gaugeTest(ServerGauge serverGauge) { addGaugeWithLabels(serverGauge, CLIENT_ID); assertGaugeExportedCorrectly(serverGauge.getGaugeName(), ExportedLabels.PARTITION_TABLENAME_TABLETYPE_KAFKATOPIC, EXPORTED_METRIC_PREFIX); + } else if (GAUGES_ACCEPTING_PARTITION_WITH_STREAM_TOPIC.contains(serverGauge)) { + addPartitionGaugeWithStreamTopicLabels(serverGauge, TABLE_NAME_WITH_TYPE); + assertGaugeExportedCorrectly(serverGauge.getGaugeName(), + ExportedLabels.PARTITIONNUM_TABLENAME_TABLETYPE_KAFKATOPIC, EXPORTED_METRIC_PREFIX); } else if (GAUGES_ACCEPTING_PARTITION.contains(serverGauge)) { addPartitionGaugeWithLabels(serverGauge, TABLE_NAME_WITH_TYPE); assertGaugeExportedCorrectly(serverGauge.getGaugeName(), ExportedLabels.PARTITION_TABLENAME_TABLETYPE, @@ -145,6 +152,10 @@ private void addPartitionGaugeWithLabels(ServerGauge serverGauge, String labels) _serverMetrics.setValueOfPartitionGauge(labels, 3, serverGauge, 100L); } + private void addPartitionGaugeWithStreamTopicLabels(ServerGauge serverGauge, String tableNameWithType) { + _serverMetrics.setValueOfTableGauge(tableNameWithType + "-" + KAFKA_TOPIC + "-3", serverGauge, 100L); + } + public void addMeterWithLabels(ServerMeter serverMeter, String labels) { _serverMetrics.addMeteredTableValue(labels, serverMeter, 4L); } From 8aad969ee56a85ed0b11ea7219c7a91bcf459210 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 18:59:34 +0530 Subject: [PATCH 07/53] Update AbstractMetricsTest.java --- .../common/metrics/AbstractMetricsTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java index c6bd8d8c30b6..e105d7ffbea3 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java @@ -456,6 +456,23 @@ public void testPartitionGauges() { Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); } + @Test + public void testPartitionGaugesWithStreamTopic() { + ControllerMetrics controllerMetrics = buildTestMetrics(); + String tableWithType = "myTable_REALTIME"; + String topic = "events"; + int partitionGroupId = 3; + String compositeKey = tableWithType + "-" + topic + "-" + partitionGroupId; + + controllerMetrics.setOrUpdatePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, + ControllerGauge.VERSION, () -> 7L); + Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, + ControllerGauge.VERSION.getGaugeName() + "." + compositeKey), 7); + + controllerMetrics.removePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, ControllerGauge.VERSION); + Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); + } + @Test public void testAddCallbackGauges() { ControllerMetrics controllerMetrics = buildTestMetrics(); From 2ca726cdd5f9a4ae16e4020baac015504a60110f Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 20:18:35 +0530 Subject: [PATCH 08/53] Update AbstractMetrics.java --- .../pinot/common/metrics/AbstractMetrics.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index e776dd0c04f3..f5ecf25d9664 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -566,6 +566,32 @@ public void setOrUpdatePartitionGauge(final String tableName, final int partitio setOrUpdateGauge(fullGaugeName, valueSupplier); } + /** + * Registers a per-partition table gauge keyed by table name with type, Kafka (or other stream) topic, and partition + * group id. The resulting MBean name matches {@code server.yml} rules so Prometheus exports {@code topic} and + * {@code partition} labels (see apache/pinot#18099). + */ + public void setOrUpdatePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, + final int partitionGroupId, final G gauge, final Supplier valueSupplier) { + Preconditions.checkArgument(StringUtils.isNotBlank(topicName), "topicName must not be blank"); + setOrUpdateTableGauge(composeStreamTopicPartitionKey(tableNameWithType, topicName, partitionGroupId), gauge, + valueSupplier); + } + + /** + * Removes a gauge registered via {@link #setOrUpdatePartitionGaugeForStreamTopic}. + */ + public void removePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, + final int partitionGroupId, final G gauge) { + Preconditions.checkArgument(StringUtils.isNotBlank(topicName), "topicName must not be blank"); + removeTableGauge(composeStreamTopicPartitionKey(tableNameWithType, topicName, partitionGroupId), gauge); + } + + private static String composeStreamTopicPartitionKey(String tableNameWithType, String topicName, + int partitionGroupId) { + return tableNameWithType + "-" + topicName + "-" + partitionGroupId; + } + /** * @deprecated please use setOrUpdateGauge(final String metricName, final Supplier valueSupplier) instead. * From 9922d54a57b54efa4bbbf38514d1545e5ae04911 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 20:19:24 +0530 Subject: [PATCH 09/53] Update AbstractMetricsTest.java --- .../org/apache/pinot/common/metrics/AbstractMetricsTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java index e105d7ffbea3..68cb70f74008 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java @@ -469,7 +469,8 @@ public void testPartitionGaugesWithStreamTopic() { Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, ControllerGauge.VERSION.getGaugeName() + "." + compositeKey), 7); - controllerMetrics.removePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, ControllerGauge.VERSION); + controllerMetrics.removePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, + ControllerGauge.VERSION); Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); } From b6fa133c0d8e5fe9df128591ec5a1213d54103bd Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Sun, 12 Apr 2026 20:20:33 +0530 Subject: [PATCH 10/53] Update PinotPrometheusMetricsTest.java --- .../common/metrics/prometheus/PinotPrometheusMetricsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java index ac7a0c415438..03b8d978ef04 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java @@ -314,7 +314,7 @@ public static class ExportedLabels { TableType.REALTIME.toString(), TOPIC, KAFKA_TOPIC); public static final List PARTITION_TABLENAME_TABLETYPE = List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString()); - + public static final List PARTITIONNUM_TABLENAME_TABLETYPE_KAFKATOPIC = List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString(), TOPIC, KAFKA_TOPIC); From 4597452836cfa6fc84c87db26f4997f0f9659fd4 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 01:10:58 +0530 Subject: [PATCH 11/53] Update IngestionDelayTracker.java --- .../core/data/manager/realtime/IngestionDelayTracker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index 907453d0b5f4..0ff7e0ea4b6b 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -337,9 +337,9 @@ public Thread newThread(Runnable r) { */ private void removePartitionId(int partitionId) { _partitionsHostedByThisServer.remove(partitionId); - _ingestionInfoMap.remove(partitionId); + IngestionInfo removed = _ingestionInfoMap.remove(partitionId); _partitionsTracked.computeIfPresent(partitionId, (k, v) -> { - removeMetrics(partitionId); + removeMetrics(partitionId, removed != null ? removed._streamTopicName : null); return null; }); From ab8ced76c9bca2ed61052ac4e953619c2aac1fab Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 01:14:56 +0530 Subject: [PATCH 12/53] Update RealtimeTableDataManager.java --- .../manager/realtime/RealtimeTableDataManager.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 5928b2f8c28f..d07242d4440a 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -310,17 +310,18 @@ protected void doShutdown() { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing * the segment name) - * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion delay metric labels + * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion + * delay metric labels * @param ingestionTimeMs ingestion time of the last consumed message (from * {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) * @param currentOffset offset of the last consumed message (from {@link StreamMessageMetadata}) */ - public void updateIngestionMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, long ingestionTimeMs, - long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { - _ingestionDelayTracker.updateMetrics(segmentName, partitionId, streamTopicName, ingestionTimeMs, firstStreamIngestionTimeMs, - currentOffset); + public void updateIngestionMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, + long ingestionTimeMs, long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { + _ingestionDelayTracker.updateMetrics(segmentName, partitionId, streamTopicName, ingestionTimeMs, + firstStreamIngestionTimeMs, currentOffset); } /** From ff5bd0996dbd799ca8441c9e8db457d6b4c31395 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 01:15:57 +0530 Subject: [PATCH 13/53] Update IngestionDelayTracker.java --- .../core/data/manager/realtime/IngestionDelayTracker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index 0ff7e0ea4b6b..12141aa4dd4d 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -430,8 +430,8 @@ private void removeMetrics(int partitionId, @Nullable String streamTopicName) { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing the * segment name) - * @param streamTopicName stream topic name for this consumer (e.g. Kafka topic); when blank, metrics use the legacy - * name without a topic label + * @param streamTopicName stream topic name for this consumer (e.g. Kafka topic); when blank, metrics use + * the legacy name without a topic label * @param ingestionTimeMs ingestion time of the last consumed message (from {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) From fc7a92d63f8d8c441eab4db457553ec6daa52efe Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 01:17:20 +0530 Subject: [PATCH 14/53] Update IngestionDelayTrackerTest.java --- .../realtime/IngestionDelayTrackerTest.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index a01fd258d6e5..07d6cadcf129 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -213,7 +213,8 @@ public void testRecordIngestionDelayWithNoAging() { // Test we follow a single partition up and down for (long ingestionTimeMs = 0; ingestionTimeMs <= maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, + firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -225,7 +226,8 @@ public void testRecordIngestionDelayWithNoAging() { // Test tracking down a measure for a given partition for (long ingestionTimeMs = maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, + firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -239,7 +241,8 @@ public void testRecordIngestionDelayWithNoAging() { // Bring up partition1 delay up and verify values for (long ingestionTimeMs = 0; ingestionTimeMs <= 2 * maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, + firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -250,7 +253,8 @@ public void testRecordIngestionDelayWithNoAging() { // Bring down values of partition1 and verify values for (long ingestionTimeMs = 2 * maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, + firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -506,8 +510,8 @@ public void testIngestionDelay() { ingestionDelayTracker._partitionsHostedByThisServer.put(partition0, true); ingestionDelayTracker._partitionsHostedByThisServer.put(partition1, true); - ingestionDelayTracker.updateMetrics(segment0, partition0, null, System.currentTimeMillis(), System.currentTimeMillis(), - new LongMsgOffset(50)); + ingestionDelayTracker.updateMetrics(segment0, partition0, null, System.currentTimeMillis(), + System.currentTimeMillis(), new LongMsgOffset(50)); ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); Map partitionIdVsLatestOffset = new HashMap<>(); From 2112cf6084e44b06abfb6e29bbdbcfd8fa64fb6f Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 01:38:57 +0530 Subject: [PATCH 15/53] Update RealtimeSegmentDataManager.java --- .../data/manager/realtime/RealtimeSegmentDataManager.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index f7d82e65a79c..cad78153120a 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -2110,7 +2110,8 @@ private void updateIngestionMetrics(StreamMessageMetadata metadata) { if (metadata != null) { try { _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, - metadata.getRecordIngestionTimeMs(), metadata.getFirstStreamRecordIngestionTimeMs(), metadata.getOffset()); + _streamConfig.getTopicName(), metadata.getRecordIngestionTimeMs(), + metadata.getFirstStreamRecordIngestionTimeMs(), metadata.getOffset()); } catch (Exception e) { _segmentLogger.warn("Failed to update the ingestion metrics", e); } @@ -2123,8 +2124,8 @@ private void updateIngestionMetrics(StreamMessageMetadata metadata) { */ private void setIngestionDelayToZero() { long currentTimeMs = System.currentTimeMillis(); - _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, currentTimeMs, currentTimeMs, - null); + _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, _streamConfig.getTopicName(), + currentTimeMs, currentTimeMs, null); } // This should be done during commit? We may not always commit when we build a segment.... From fb55606439b968de505834d1d87e3e3fc71612aa Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 02:03:38 +0530 Subject: [PATCH 16/53] Update RealtimeTableDataManager.java --- .../core/data/manager/realtime/RealtimeTableDataManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index d07242d4440a..8ff747fe39e6 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -310,7 +310,7 @@ protected void doShutdown() { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing * the segment name) - * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion + * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion * delay metric labels * @param ingestionTimeMs ingestion time of the last consumed message (from * {@link StreamMessageMetadata}) From dfeadfa2d7f9a7a1d05cad0447a9ce73fd69ef6c Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 02:14:29 +0530 Subject: [PATCH 17/53] Update IngestionDelayTrackerTest.java --- .../manager/realtime/IngestionDelayTrackerTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index 07d6cadcf129..1e84b433a168 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -314,7 +314,7 @@ public void testRecordIngestionDelayWithAging() { (partition0Delay1 + partition0Offset1Ms)); ingestionTimeMs = offsetClock.millis() - partition1Delay0; - ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), partition1Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), partition1Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition1), ingestionTimeMs); @@ -540,17 +540,17 @@ public void testIngestionDelay() { @Test public void testStreamTopicRegistersTopicTaggedGauges() { - IngestionDelayTracker tracker = createTracker(); + IngestionDelayTracker ingestionDelayTracker = createTracker(); Instant now = Instant.now(); - tracker.setClock(Clock.fixed(now, ZoneId.systemDefault())); + ingestionDelayTracker.setClock(Clock.fixed(now, ZoneId.systemDefault())); String segment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); long ts = now.toEpochMilli(); - tracker.updateIngestionMetrics(segment, 0, "test_topic", ts, ts, null, null); + ingestionDelayTracker.updateMetrics(segment, 0, "test_topic", ts, ts, null, null); verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), eq(ServerGauge.REALTIME_INGESTION_DELAY_MS), any()); verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), eq(ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS), any()); - tracker.shutdown(); + ingestionDelayTracker.shutdown(); } private void verifyMetrics(Map>> partitionToMetricToValues) { From 8e239e93134d96996fe7f36f0dfff86dc57fddb8 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 02:17:36 +0530 Subject: [PATCH 18/53] Update IngestionDelayTrackerTest.java --- .../core/data/manager/realtime/IngestionDelayTrackerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index 1e84b433a168..6b89eadb19ec 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -545,7 +545,7 @@ public void testStreamTopicRegistersTopicTaggedGauges() { ingestionDelayTracker.setClock(Clock.fixed(now, ZoneId.systemDefault())); String segment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); long ts = now.toEpochMilli(); - ingestionDelayTracker.updateMetrics(segment, 0, "test_topic", ts, ts, null, null); + ingestionDelayTracker.updateMetrics(segment, 0, "test_topic", ts, ts, null); verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), eq(ServerGauge.REALTIME_INGESTION_DELAY_MS), any()); verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), From 264cbbb03f5090f5af072b810f023ad5b111f1f2 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 13 Apr 2026 02:27:31 +0530 Subject: [PATCH 19/53] Update IngestionDelayTrackerTest.java --- .../core/data/manager/realtime/IngestionDelayTrackerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index 6b89eadb19ec..a67b97568a7e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -53,8 +53,8 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class IngestionDelayTrackerTest { From aaf74b1a6116085c9b7c4504b7a0527fffd91b7d Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 14 Apr 2026 00:01:28 +0530 Subject: [PATCH 20/53] Update IngestionDelayTrackerTest.java --- .../data/manager/realtime/IngestionDelayTrackerTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index a67b97568a7e..b76237bdb9e2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -546,10 +546,9 @@ public void testStreamTopicRegistersTopicTaggedGauges() { String segment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); long ts = now.toEpochMilli(); ingestionDelayTracker.updateMetrics(segment, 0, "test_topic", ts, ts, null); - verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), - eq(ServerGauge.REALTIME_INGESTION_DELAY_MS), any()); - verify(_serverMetrics).setOrUpdatePartitionGaugeForStreamTopic(eq(REALTIME_TABLE_NAME), eq("test_topic"), eq(0), - eq(ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS), any()); + Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), ts); + Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(0).longValue(), 0L); + Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0).longValue(), 0L); ingestionDelayTracker.shutdown(); } From 5e0819cc97efc46c7094e3ccb0d81505214dddba Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 14 Apr 2026 21:26:59 +0530 Subject: [PATCH 21/53] Update IngestionDelayTracker.java --- .../manager/realtime/IngestionDelayTracker.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index 12141aa4dd4d..b26abdfabac3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -388,10 +388,19 @@ void createMetrics(int partitionId) { _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, () -> getLatestPartitionOffset(partitionId)); } - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_MS, - () -> getPartitionIngestionDelayMs(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, - ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); + IngestionInfo ingestionInfo = _ingestionInfoMap.get(partitionId); + @Nullable String streamTopicName = ingestionInfo != null ? ingestionInfo._streamTopicName : null; + if (StringUtils.isNotBlank(streamTopicName)) { + _serverMetrics.setOrUpdatePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, + ServerGauge.REALTIME_INGESTION_DELAY_MS, () -> getPartitionIngestionDelayMs(partitionId)); + _serverMetrics.setOrUpdatePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, + ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); + } else { + _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_MS, + () -> getPartitionIngestionDelayMs(partitionId)); + _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, + ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); + } _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS, () -> getPartitionIngestionReportingStatus(partitionId)); From 7a870404c06f3137424ccb72a503d798b6c02c78 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 14 Apr 2026 21:28:01 +0530 Subject: [PATCH 22/53] Update IngestionDelayTrackerTest.java --- .../core/data/manager/realtime/IngestionDelayTrackerTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index b76237bdb9e2..c5f4e1ac79aa 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -50,10 +50,7 @@ import org.testng.Assert; import org.testng.annotations.Test; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; From 5f89f3ee8f0f61bcbd8fce3ef3092091ac758c41 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 20 Apr 2026 11:05:12 +0530 Subject: [PATCH 23/53] Update AbstractMetrics.java --- .../pinot/common/metrics/AbstractMetrics.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index f5ecf25d9664..5ef6e3d3e64f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -566,10 +566,25 @@ public void setOrUpdatePartitionGauge(final String tableName, final int partitio setOrUpdateGauge(fullGaugeName, valueSupplier); } + /** + * Same {@code gauge} and exported Prometheus metric name as the per-partition overload without a topic when + * {@code streamTopicName} is null or blank. When a topic is provided, registers using the composite JMX key + * expected by {@code server.yml} so the same metric gains {@code topic} and {@code partition} labels (not a separate + * gauge enum value or metric family—only the MBean naming differs for the exporter). + */ + public void setOrUpdatePartitionGauge(final String tableName, final int partitionId, + @Nullable final String streamTopicName, final G gauge, final Supplier valueSupplier) { + if (StringUtils.isNotBlank(streamTopicName)) { + setOrUpdatePartitionGaugeForStreamTopic(tableName, streamTopicName, partitionId, gauge, valueSupplier); + } else { + setOrUpdatePartitionGauge(tableName, partitionId, gauge, valueSupplier); + } + } + /** * Registers a per-partition table gauge keyed by table name with type, Kafka (or other stream) topic, and partition * group id. The resulting MBean name matches {@code server.yml} rules so Prometheus exports {@code topic} and - * {@code partition} labels (see apache/pinot#18099). + * {@code partition} labels on the same metric series as the table+partition-only registration (see apache/pinot#18099). */ public void setOrUpdatePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, final int partitionGroupId, final G gauge, final Supplier valueSupplier) { @@ -787,6 +802,18 @@ public void removePartitionGauge(final String tableName, final int partitionId, removeGauge(fullGaugeName); } + /** + * Counterpart to the per-partition overload that takes an optional {@code streamTopicName}. + */ + public void removePartitionGauge(final String tableName, final int partitionId, + @Nullable final String streamTopicName, final G gauge) { + if (StringUtils.isNotBlank(streamTopicName)) { + removePartitionGaugeForStreamTopic(tableName, streamTopicName, partitionId, gauge); + } else { + removePartitionGauge(tableName, partitionId, gauge); + } + } + /** * Removes a table gauge given the table name, the key and the gauge. * The add/remove is expected to work correctly in case of being invoked across multiple threads. From e32e45d44929174ed7080f6870f8682c7c8f8e08 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 20 Apr 2026 11:10:24 +0530 Subject: [PATCH 24/53] Update IngestionDelayTracker.java --- .../data/manager/realtime/IngestionDelayTracker.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index b26abdfabac3..2a0acd79bc81 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -390,17 +390,10 @@ void createMetrics(int partitionId) { } IngestionInfo ingestionInfo = _ingestionInfoMap.get(partitionId); @Nullable String streamTopicName = ingestionInfo != null ? ingestionInfo._streamTopicName : null; - if (StringUtils.isNotBlank(streamTopicName)) { - _serverMetrics.setOrUpdatePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, + _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, streamTopicName, ServerGauge.REALTIME_INGESTION_DELAY_MS, () -> getPartitionIngestionDelayMs(partitionId)); - _serverMetrics.setOrUpdatePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, - ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); - } else { - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_MS, - () -> getPartitionIngestionDelayMs(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, + _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, streamTopicName, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); - } _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS, () -> getPartitionIngestionReportingStatus(partitionId)); From 23630e0ffb39597bc40351fbe8f52353c05b219c Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 20 Apr 2026 11:12:48 +0530 Subject: [PATCH 25/53] Update IngestionDelayTracker.java --- .../data/manager/realtime/IngestionDelayTracker.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index 2a0acd79bc81..df158b89dba9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -411,17 +411,12 @@ private void removeMetrics(int partitionId, @Nullable String streamTopicName) { _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); } - _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_MS); - _serverMetrics.removePartitionGauge(_metricName, partitionId, + _serverMetrics.removePartitionGauge(_metricName, partitionId, streamTopicName, + ServerGauge.REALTIME_INGESTION_DELAY_MS); + _serverMetrics.removePartitionGauge(_metricName, partitionId, streamTopicName, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS); - if (StringUtils.isNotBlank(streamTopicName)) { - _serverMetrics.removePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, - ServerGauge.REALTIME_INGESTION_DELAY_MS); - _serverMetrics.removePartitionGaugeForStreamTopic(_metricName, streamTopicName, partitionId, - ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); - } LOGGER.info("Successfully removed ingestion metrics for partition id: {}", partitionId); } From 128854d97c0aff9e2a8a91447a2a1713fa46d4e7 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Mon, 20 Apr 2026 11:16:16 +0530 Subject: [PATCH 26/53] Update AbstractMetricsTest.java --- .../common/metrics/AbstractMetricsTest.java | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java index 68cb70f74008..88d87bce9d22 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java @@ -474,6 +474,37 @@ public void testPartitionGaugesWithStreamTopic() { Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); } + @Test + public void testPartitionGaugeWithOptionalStreamTopicDelegates() { + ControllerMetrics controllerMetrics = buildTestMetrics(); + String tableWithType = "myTable_REALTIME"; + String topic = "events"; + int partitionGroupId = 3; + String compositeKey = tableWithType + "-" + topic + "-" + partitionGroupId; + + controllerMetrics.setOrUpdatePartitionGauge(tableWithType, partitionGroupId, topic, ControllerGauge.VERSION, + () -> 7L); + Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, + ControllerGauge.VERSION.getGaugeName() + "." + compositeKey), 7); + + controllerMetrics.removePartitionGauge(tableWithType, partitionGroupId, topic, ControllerGauge.VERSION); + Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); + } + + @Test + public void testPartitionGaugeWithBlankOptionalStreamTopicMatchesPartitionOnly() { + ControllerMetrics controllerMetrics = buildTestMetrics(); + String table = "test_table"; + int partitionId = 1024; + + controllerMetrics.setOrUpdatePartitionGauge(table, partitionId, null, ControllerGauge.VERSION, () -> 2L); + Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, + ControllerGauge.VERSION.getGaugeName() + "." + table + "." + partitionId), 2); + + controllerMetrics.removePartitionGauge(table, partitionId, "", ControllerGauge.VERSION); + Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); + } + @Test public void testAddCallbackGauges() { ControllerMetrics controllerMetrics = buildTestMetrics(); From 71abbd02cdaf10d9757014324068329b22fdf7bf Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:25:56 +0530 Subject: [PATCH 27/53] Update IngestionConfigUtils.java --- .../pinot/spi/utils/IngestionConfigUtils.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java index 40f647c429cd..bd99dfd5fa43 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java @@ -141,6 +141,20 @@ public static int getStreamConfigIndexFromPinotPartitionId(int partitionId) { return partitionId / PARTITION_PADDING_OFFSET; } + /** + * Returns the stream consumer client id without the optional instance suffix from + * {@link org.apache.pinot.spi.config.instance.InstanceDataManagerConfig#getConsumerClientIdSuffix()}. + * Format: {@code tableNameWithType + "-" + streamTopic + "-" + partitionId}. + *

Use the stream topic partition id for {@code partitionId} when matching realtime consumer client ids. Other + * callers (for example stream-aware ingestion gauges) may pass the Pinot partition group id as the third segment; + * the string format matches server metrics keys that {@code server.yml} maps to Prometheus series with + * {@code topic} and {@code partition} labels. + */ + public static String getStreamConsumerClientIdWithoutSuffix(String tableNameWithType, String streamTopic, + int partitionId) { + return tableNameWithType + "-" + streamTopic + "-" + partitionId; + } + /** * Fetches the streamConfig from the list of streamConfigs according to the partition id. */ From 0c11831959864bfda96c707d41b49dd6e514f025 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:26:52 +0530 Subject: [PATCH 28/53] Update IngestionConfigUtilsTest.java --- .../apache/pinot/spi/utils/IngestionConfigUtilsTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java index 3390126b2ffb..c306a140b1b0 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java @@ -182,4 +182,10 @@ public void testGetStreamConfigIndexToStreamPartitions() { Assert.assertEquals(streamConfigIndexToStreamPartitions.get(1), new HashSet<>(Arrays.asList(100, 1))); Assert.assertEquals(streamConfigIndexToStreamPartitions.get(3), new HashSet<>(Arrays.asList(400))); } + + @Test + public void testGetStreamConsumerClientIdWithoutSuffix() { + Assert.assertEquals(IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix( + "db.myTable_REALTIME", "events", 3), "db.myTable_REALTIME-events-3"); + } } From 1a31661a5b9979c44738fce8367b045c1619ec0b Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:35:53 +0530 Subject: [PATCH 29/53] Update RealtimeSegmentDataManager.java --- .../realtime/RealtimeSegmentDataManager.java | 65 ++++++++++--------- 1 file changed, 34 insertions(+), 31 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 586fe0d00031..db71644df89f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -315,6 +315,7 @@ public void deleteSegmentFile() { private final int _streamPartitionId; private final PartitionGroupConsumptionStatus _partitionGroupConsumptionStatus; final String _clientId; + private final String _streamConsumerMetricBaseKey; private final TransformPipeline _transformPipeline; private PartitionGroupConsumer _partitionGroupConsumer = null; private StreamMetadataProvider _partitionMetadataProvider = null; @@ -483,14 +484,14 @@ protected boolean consumeLoop() _segmentLogger.info("Starting consumption loop start offset {}, finalOffset {}", _currentOffset, _finalOffset); while (!_shouldStop && !endCriteriaReached()) { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 1); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 1); // Consume for the next readTime ms, or we get to final offset, whichever happens earlier, // Update _currentOffset upon return from this method MessageBatch messageBatch; try { messageBatch = _partitionGroupConsumer.fetchMessages(_currentOffset, _streamConfig.getFetchTimeoutMillis()); //track realtime rows fetched on a table level. This included valid + invalid rows - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FETCHED, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_FETCHED, messageBatch.getUnfilteredMessageCount()); if (_segmentLogger.isDebugEnabled()) { _segmentLogger.debug("message batch received. filtered={} unfiltered={} endOfPartitionGroup={}", @@ -531,7 +532,7 @@ protected boolean consumeLoop() // TODO Issue 5359 Need to find a way to bump metrics without getting actual offset value. if (_currentOffset instanceof LongMsgOffset) { // TODO: only LongMsgOffset supplies long offset value. - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.HIGHEST_STREAM_OFFSET_CONSUMED, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.HIGHEST_STREAM_OFFSET_CONSUMED, ((LongMsgOffset) _currentOffset).getOffset()); } lastUpdatedOffset = _streamPartitionMsgOffsetFactory.create(_currentOffset); @@ -572,11 +573,11 @@ protected boolean consumeLoop() } if (_numRowsErrored > 0) { - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); _serverMetrics.addMeteredTableValue(_tableStreamName, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); // TODO Although the metric is called real-time, updating it at this point is not really real-time. The choice of // name is partly to avoid a more convoluted name and partly in anticipation of making it real-time. - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_BYTES_DROPPED, _numBytesDropped); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_DROPPED, _numBytesDropped); } return true; } @@ -657,7 +658,7 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } // Silently drop the row with error realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, realtimeRowsDroppedMeter); _numRowsErrored++; _numBytesDropped += rowSizeInBytes; @@ -677,7 +678,7 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee if (result != null) { if (result.getSkippedRowCount() > 0) { realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_FILTERED, result.getSkippedRowCount(), realtimeRowsDroppedMeter); if (_trackFilteredMessageOffsets) { _filteredMessageOffsets.add(offset.toString()); @@ -685,12 +686,12 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } if (result.getIncompleteRowCount() > 0) { realtimeIncompleteRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, result.getIncompleteRowCount(), realtimeIncompleteRowsConsumedMeter); } if (result.getSanitizedRowCount() > 0) { realtimeRowsSanitizedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_SANITIZED, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_SANITIZED, result.getSanitizedRowCount(), realtimeRowsSanitizedMeter); } List transformedRows = result.getTransformedRows(); @@ -701,14 +702,14 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee _lastRowMetadata = metadata; _lastConsumedTimestampMs = System.currentTimeMillis(); realtimeRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_CONSUMED, 1, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_CONSUMED, 1, realtimeRowsConsumedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_ROWS_CONSUMED, 1L); int recordSerializedValueLength = _lastRowMetadata.getRecordSerializedSize(); if (recordSerializedValueLength > 0) { realtimeBytesIngestedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_BYTES_CONSUMED, + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength, realtimeBytesIngestedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength); } @@ -833,20 +834,20 @@ public void run() { if (_state.shouldConsume()) { consumeLoop(); // Consume until we reached the end criteria, or we are stopped. } - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); if (_shouldStop) { break; } if (_state == State.INITIAL_CONSUMING) { initialConsumptionEnd = now(); - _serverMetrics.setValueOfTableGauge(_clientId, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_INITIAL_CONSUMPTION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(initialConsumptionEnd - _startTimeMs)); } else if (_state == State.CATCHING_UP) { catchUpTimeMillis += now() - lastCatchUpStart; _serverMetrics - .setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, + .setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(catchUpTimeMillis)); } @@ -943,7 +944,7 @@ public void run() { // respectively. // Refer to the PR for the new commit protocol: https://github.com/apache/pinot/pull/14741 if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); if (!startSegmentCommit()) { // If for any reason commit failed, we don't want to be in COMMITTING state when we hold. // Change the state to HOLDING before looping around. @@ -953,7 +954,7 @@ public void run() { break; } } else { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); } long buildTimeSeconds = response.getBuildTimeSeconds(); try { @@ -1002,7 +1003,7 @@ public void run() { _segmentLogger.error(errorMessage, e); _state = State.ERROR; _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); postStopConsumedMsg(e.getClass().getName()); return; } @@ -1012,7 +1013,7 @@ public void run() { if (initialConsumptionEnd != 0L) { _serverMetrics - .setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, + .setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(now() - initialConsumptionEnd)); } // There is a race condition that the destroy() method can be called which ends up calling stop on the consumer. @@ -1022,7 +1023,7 @@ public void run() { // so it is ok not to mark it non-consuming, as the main thread will clean up this metric in destroy() method // as the final step. if (!_shouldStop) { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); } } @@ -1260,9 +1261,9 @@ protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) } long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(buildTimeMillis)); - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, TimeUnit.MILLISECONDS.toSeconds(waitTimeMillis)); if (forCommit) { @@ -1313,7 +1314,7 @@ protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) private void reportSegmentBuildFailure(String errorMessage, @Nullable Exception e) { _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.SEGMENT_BUILD_FAILURE, 1); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.SEGMENT_BUILD_FAILURE, 1); } @VisibleForTesting @@ -1428,10 +1429,10 @@ private void releaseConsumerSemaphore() { * which no longer resides in this host any more, thus causes false positive information to the metric system. */ private void cleanupMetrics() { - _serverMetrics.removeTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING); - _serverMetrics.removeTableGauge(_clientId, ServerGauge.STREAM_DATA_LOSS); - _serverMetrics.removeTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED); - _serverMetrics.removeTableMeter(_clientId, ServerMeter.SEGMENT_BUILD_FAILURE); + _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING); + _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.STREAM_DATA_LOSS); + _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED); + _serverMetrics.removeTableMeter(_streamConsumerMetricBaseKey, ServerMeter.SEGMENT_BUILD_FAILURE); } protected void hold() @@ -1507,7 +1508,7 @@ private void removeSegmentFile() { public void goOnlineFromConsuming(SegmentZKMetadata segmentZKMetadata) throws InterruptedException { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); try { // Remove the segment file before we do anything else. removeSegmentFile(); @@ -1625,7 +1626,7 @@ public void goOnlineFromConsuming(SegmentZKMetadata segmentZKMetadata) } catch (Exception e) { Utils.rethrowException(e); } finally { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); } } @@ -1656,7 +1657,7 @@ private boolean catchupToFinalOffset(StreamPartitionMsgOffset endOffset, long ti _segmentLogger.warn("Exception when catching up to final offset", e); return false; } finally { - _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); } if (_currentOffset.compareTo(endOffset) != 0) { // Timeout? @@ -1793,6 +1794,8 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf } else { _clientId = _tableNameWithType + "-" + streamTopic + "-" + _streamPartitionId; } + _streamConsumerMetricBaseKey = + IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix(_tableNameWithType, streamTopic, _streamPartitionId); _segmentLogger = LoggerFactory.getLogger(RealtimeSegmentDataManager.class.getName() + "_" + _segmentNameStr); _tableStreamName = _tableNameWithType + "_" + streamTopic; if (indexLoadingConfig.isRealtimeOffHeapAllocation() && !indexLoadingConfig.isDirectRealtimeOffHeapAllocation()) { @@ -2070,7 +2073,7 @@ private void makeStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to create stream consumer for topic partition {} reason {}", _clientId, reason, e); - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); throw e; } } @@ -2091,7 +2094,7 @@ private void recreateStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to recreate stream consumer for topic partition {}", _clientId, e); - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); throw e; } } From a29fa6a926e2a164cd65816db1469f9481fea3f3 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:37:17 +0530 Subject: [PATCH 30/53] Update AbstractMetrics.java --- .../java/org/apache/pinot/common/metrics/AbstractMetrics.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index 3d542558b7dd..d70c41f1fad9 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -40,6 +40,7 @@ import org.apache.pinot.spi.metrics.PinotMetricUtils; import org.apache.pinot.spi.metrics.PinotMetricsRegistry; import org.apache.pinot.spi.metrics.PinotTimer; +import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -608,7 +609,8 @@ public void removePartitionGaugeForStreamTopic(final String tableNameWithType, f private static String composeStreamTopicPartitionKey(String tableNameWithType, String topicName, int partitionGroupId) { - return tableNameWithType + "-" + topicName + "-" + partitionGroupId; + return IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix(tableNameWithType, topicName, + partitionGroupId); } /** From 647554dda53fe6169d7f75dfad2a445c9a56f1e0 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:50:05 +0530 Subject: [PATCH 31/53] Update AbstractMetrics.java --- .../java/org/apache/pinot/common/metrics/AbstractMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index d70c41f1fad9..5331594105bc 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -589,7 +589,7 @@ public void setOrUpdatePartitionGauge(final String tableName, final int partitio /** * Registers a per-partition table gauge keyed by table name with type, Kafka (or other stream) topic, and partition * group id. The resulting MBean name matches {@code server.yml} rules so Prometheus exports {@code topic} and - * {@code partition} labels on the same metric series as the table+partition-only registration (see apache/pinot#18099). + * {@code partition} labels on the same metric series as the table+partition-only registration (see #18099). */ public void setOrUpdatePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, final int partitionGroupId, final G gauge, final Supplier valueSupplier) { From fe0d33c2832143beb310d2926821933a6936a323 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 18:55:24 +0530 Subject: [PATCH 32/53] Update RealtimeSegmentDataManager.java --- .../realtime/RealtimeSegmentDataManager.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index db71644df89f..db7c9bd612c3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -577,7 +577,8 @@ protected boolean consumeLoop() _serverMetrics.addMeteredTableValue(_tableStreamName, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); // TODO Although the metric is called real-time, updating it at this point is not really real-time. The choice of // name is partly to avoid a more convoluted name and partly in anticipation of making it real-time. - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_DROPPED, _numBytesDropped); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_DROPPED, + _numBytesDropped); } return true; } @@ -658,8 +659,8 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } // Silently drop the row with error realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, - realtimeRowsDroppedMeter); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, + 1, realtimeRowsDroppedMeter); _numRowsErrored++; _numBytesDropped += rowSizeInBytes; } @@ -686,8 +687,9 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } if (result.getIncompleteRowCount() > 0) { realtimeIncompleteRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, - result.getIncompleteRowCount(), realtimeIncompleteRowsConsumedMeter); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, + ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, result.getIncompleteRowCount(), + realtimeIncompleteRowsConsumedMeter); } if (result.getSanitizedRowCount() > 0) { realtimeRowsSanitizedMeter = @@ -702,8 +704,8 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee _lastRowMetadata = metadata; _lastConsumedTimestampMs = System.currentTimeMillis(); realtimeRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_CONSUMED, 1, - realtimeRowsConsumedMeter); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_CONSUMED, + 1, realtimeRowsConsumedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_ROWS_CONSUMED, 1L); int recordSerializedValueLength = _lastRowMetadata.getRecordSerializedSize(); From bb45a22f6554f1b406c38d94af82ac4379dfcdec Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 19:02:15 +0530 Subject: [PATCH 33/53] Update RealtimeSegmentDataManager.java --- .../realtime/RealtimeSegmentDataManager.java | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index db7c9bd612c3..23a9e995cbfe 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -659,8 +659,8 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } // Silently drop the row with error realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, - 1, realtimeRowsDroppedMeter); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, + ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, realtimeRowsDroppedMeter); _numRowsErrored++; _numBytesDropped += rowSizeInBytes; } @@ -711,8 +711,8 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee int recordSerializedValueLength = _lastRowMetadata.getRecordSerializedSize(); if (recordSerializedValueLength > 0) { realtimeBytesIngestedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_CONSUMED, - recordSerializedValueLength, realtimeBytesIngestedMeter); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, + ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength, realtimeBytesIngestedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength); } } catch (Exception e) { @@ -849,7 +849,8 @@ public void run() { } else if (_state == State.CATCHING_UP) { catchUpTimeMillis += now() - lastCatchUpStart; _serverMetrics - .setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, + .setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(catchUpTimeMillis)); } @@ -946,7 +947,8 @@ public void run() { // respectively. // Refer to the PR for the new commit protocol: https://github.com/apache/pinot/pull/14741 if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, + 1); if (!startSegmentCommit()) { // If for any reason commit failed, we don't want to be in COMMITTING state when we hold. // Change the state to HOLDING before looping around. @@ -956,7 +958,8 @@ public void run() { break; } } else { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, + 0); } long buildTimeSeconds = response.getBuildTimeSeconds(); try { @@ -1015,7 +1018,8 @@ public void run() { if (initialConsumptionEnd != 0L) { _serverMetrics - .setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, + .setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(now() - initialConsumptionEnd)); } // There is a race condition that the destroy() method can be called which ends up calling stop on the consumer. @@ -1263,9 +1267,11 @@ protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) } long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(buildTimeMillis)); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, TimeUnit.MILLISECONDS.toSeconds(waitTimeMillis)); if (forCommit) { @@ -2075,7 +2081,8 @@ private void makeStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to create stream consumer for topic partition {} reason {}", _clientId, reason, e); - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, + 1L); throw e; } } @@ -2096,7 +2103,8 @@ private void recreateStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to recreate stream consumer for topic partition {}", _clientId, e); - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); + _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, + 1L); throw e; } } From 7fc4a19e58a92a5e98c6ed16a51c68788cfde2b3 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 19:02:49 +0530 Subject: [PATCH 34/53] Update IngestionDelayTracker.java --- .../pinot/core/data/manager/realtime/IngestionDelayTracker.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index df158b89dba9..a56701fa385e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -35,7 +35,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import javax.annotation.Nullable; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.metrics.ServerGauge; import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; From 1d6b8a52b141c222236d12687170e71954161981 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Fri, 1 May 2026 21:55:34 +0530 Subject: [PATCH 35/53] Update RealtimeSegmentDataManager.java --- .../data/manager/realtime/RealtimeSegmentDataManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 23a9e995cbfe..823755dc3043 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -947,8 +947,8 @@ public void run() { // respectively. // Refer to the PR for the new commit protocol: https://github.com/apache/pinot/pull/14741 if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, - 1); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); if (!startSegmentCommit()) { // If for any reason commit failed, we don't want to be in COMMITTING state when we hold. // Change the state to HOLDING before looping around. @@ -958,8 +958,8 @@ public void run() { break; } } else { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, - 0); + _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, + ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); } long buildTimeSeconds = response.getBuildTimeSeconds(); try { From 5a5e39269f18454ad439da7f3ec690b8f534746a Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 19:54:19 +0530 Subject: [PATCH 36/53] Update AbstractMetrics.java --- .../pinot/common/metrics/AbstractMetrics.java | 57 ------------------- 1 file changed, 57 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index 5331594105bc..05f4321c2c30 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -32,7 +32,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.Utils; import org.apache.pinot.spi.metrics.PinotGauge; import org.apache.pinot.spi.metrics.PinotMeter; @@ -40,7 +39,6 @@ import org.apache.pinot.spi.metrics.PinotMetricUtils; import org.apache.pinot.spi.metrics.PinotMetricsRegistry; import org.apache.pinot.spi.metrics.PinotTimer; -import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -571,48 +569,6 @@ public void setOrUpdatePartitionGauge(final String tableName, final int partitio setOrUpdateGauge(fullGaugeName, valueSupplier); } - /** - * Same {@code gauge} and exported Prometheus metric name as the per-partition overload without a topic when - * {@code streamTopicName} is null or blank. When a topic is provided, registers using the composite JMX key - * expected by {@code server.yml} so the same metric gains {@code topic} and {@code partition} labels (not a separate - * gauge enum value or metric family—only the MBean naming differs for the exporter). - */ - public void setOrUpdatePartitionGauge(final String tableName, final int partitionId, - @Nullable final String streamTopicName, final G gauge, final Supplier valueSupplier) { - if (StringUtils.isNotBlank(streamTopicName)) { - setOrUpdatePartitionGaugeForStreamTopic(tableName, streamTopicName, partitionId, gauge, valueSupplier); - } else { - setOrUpdatePartitionGauge(tableName, partitionId, gauge, valueSupplier); - } - } - - /** - * Registers a per-partition table gauge keyed by table name with type, Kafka (or other stream) topic, and partition - * group id. The resulting MBean name matches {@code server.yml} rules so Prometheus exports {@code topic} and - * {@code partition} labels on the same metric series as the table+partition-only registration (see #18099). - */ - public void setOrUpdatePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, - final int partitionGroupId, final G gauge, final Supplier valueSupplier) { - Preconditions.checkArgument(StringUtils.isNotBlank(topicName), "topicName must not be blank"); - setOrUpdateTableGauge(composeStreamTopicPartitionKey(tableNameWithType, topicName, partitionGroupId), gauge, - valueSupplier); - } - - /** - * Removes a gauge registered via {@link #setOrUpdatePartitionGaugeForStreamTopic}. - */ - public void removePartitionGaugeForStreamTopic(final String tableNameWithType, final String topicName, - final int partitionGroupId, final G gauge) { - Preconditions.checkArgument(StringUtils.isNotBlank(topicName), "topicName must not be blank"); - removeTableGauge(composeStreamTopicPartitionKey(tableNameWithType, topicName, partitionGroupId), gauge); - } - - private static String composeStreamTopicPartitionKey(String tableNameWithType, String topicName, - int partitionGroupId) { - return IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix(tableNameWithType, topicName, - partitionGroupId); - } - /** * @deprecated please use setOrUpdateGauge(final String metricName, final Supplier valueSupplier) instead. * @@ -795,7 +751,6 @@ public void removeTableGauge(final String tableName, final G gauge) { removeGauge(fullGaugeName); } - /** * Removes a table gauge given the table name, the partition id and the gauge. * The add/remove is expected to work correctly in case of being invoked across multiple threads. @@ -808,18 +763,6 @@ public void removePartitionGauge(final String tableName, final int partitionId, removeGauge(fullGaugeName); } - /** - * Counterpart to the per-partition overload that takes an optional {@code streamTopicName}. - */ - public void removePartitionGauge(final String tableName, final int partitionId, - @Nullable final String streamTopicName, final G gauge) { - if (StringUtils.isNotBlank(streamTopicName)) { - removePartitionGaugeForStreamTopic(tableName, streamTopicName, partitionId, gauge); - } else { - removePartitionGauge(tableName, partitionId, gauge); - } - } - /** * Removes a table gauge given the table name, the key and the gauge. * The add/remove is expected to work correctly in case of being invoked across multiple threads. From 09fffac593cd21755ae221ea4bd82779da152526 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 19:55:03 +0530 Subject: [PATCH 37/53] Update PinotPrometheusMetricsTest.java --- .../common/metrics/prometheus/PinotPrometheusMetricsTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java index b791a43d5dc7..0f546289ec83 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PinotPrometheusMetricsTest.java @@ -316,10 +316,6 @@ public static class ExportedLabels { public static final List PARTITION_TABLENAME_TABLETYPE = List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString()); - public static final List PARTITIONNUM_TABLENAME_TABLETYPE_KAFKATOPIC = - List.of(PARTITION, "3", TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TableType.REALTIME.toString(), TOPIC, - KAFKA_TOPIC); - public static final List TABLENAME_TABLETYPE_CONTROLLER_TASKTYPE = List.of(TABLE, ExportedLabelValues.TABLENAME, TABLETYPE, TABLETYPE_REALTIME, TASKTYPE, CONTROLLER_PERIODIC_TASK_CHC); From 5d24d7e4225b1b8b3144f7ad3a584948251dfbed Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 19:56:40 +0530 Subject: [PATCH 38/53] Update ServerPrometheusMetricsTest.java --- .../prometheus/ServerPrometheusMetricsTest.java | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java index 31de180cc203..79fb88f56ed7 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java @@ -54,12 +54,9 @@ public abstract class ServerPrometheusMetricsTest extends PinotPrometheusMetrics private static final List GAUGES_ACCEPTING_PARTITION = List.of(ServerGauge.UPSERT_VALID_DOC_ID_SNAPSHOT_COUNT, ServerGauge.UPSERT_PRIMARY_KEYS_IN_SNAPSHOT_COUNT, - ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, - ServerGauge.DEDUP_PRIMARY_KEYS_COUNT, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, - ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); - - private static final List GAUGES_ACCEPTING_PARTITION_WITH_STREAM_TOPIC = - List.of(ServerGauge.REALTIME_INGESTION_DELAY_MS, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); + ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.REALTIME_INGESTION_DELAY_MS, + ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT, + ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); private static final List GAUGES_ACCEPTING_RAW_TABLE_NAME = List.of(ServerGauge.REALTIME_OFFHEAP_MEMORY_USED, ServerGauge.REALTIME_SEGMENT_NUM_PARTITIONS, @@ -125,10 +122,6 @@ public void gaugeTest(ServerGauge serverGauge) { addGaugeWithLabels(serverGauge, CLIENT_ID); assertGaugeExportedCorrectly(serverGauge.getGaugeName(), ExportedLabels.PARTITION_TABLENAME_TABLETYPE_KAFKATOPIC, EXPORTED_METRIC_PREFIX); - } else if (GAUGES_ACCEPTING_PARTITION_WITH_STREAM_TOPIC.contains(serverGauge)) { - addPartitionGaugeWithStreamTopicLabels(serverGauge, TABLE_NAME_WITH_TYPE); - assertGaugeExportedCorrectly(serverGauge.getGaugeName(), - ExportedLabels.PARTITIONNUM_TABLENAME_TABLETYPE_KAFKATOPIC, EXPORTED_METRIC_PREFIX); } else if (GAUGES_ACCEPTING_PARTITION.contains(serverGauge)) { addPartitionGaugeWithLabels(serverGauge, TABLE_NAME_WITH_TYPE); assertGaugeExportedCorrectly(serverGauge.getGaugeName(), ExportedLabels.PARTITION_TABLENAME_TABLETYPE, @@ -152,10 +145,6 @@ private void addPartitionGaugeWithLabels(ServerGauge serverGauge, String labels) _serverMetrics.setValueOfPartitionGauge(labels, 3, serverGauge, 100L); } - private void addPartitionGaugeWithStreamTopicLabels(ServerGauge serverGauge, String tableNameWithType) { - _serverMetrics.setValueOfTableGauge(tableNameWithType + "-" + KAFKA_TOPIC + "-3", serverGauge, 100L); - } - public void addMeterWithLabels(ServerMeter serverMeter, String labels) { _serverMetrics.addMeteredTableValue(labels, serverMeter, 4L); } From ef97f39af39ae79ae55dbd5ee6dd18c4e969c76c Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 19:57:29 +0530 Subject: [PATCH 39/53] Update AbstractMetricsTest.java --- .../common/metrics/AbstractMetricsTest.java | 49 ------------------- 1 file changed, 49 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java index 32676afd4a71..6ff9c4e2331a 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java @@ -407,55 +407,6 @@ public void testPartitionGauges() { Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); } - @Test - public void testPartitionGaugesWithStreamTopic() { - ControllerMetrics controllerMetrics = buildTestMetrics(); - String tableWithType = "myTable_REALTIME"; - String topic = "events"; - int partitionGroupId = 3; - String compositeKey = tableWithType + "-" + topic + "-" + partitionGroupId; - - controllerMetrics.setOrUpdatePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, - ControllerGauge.VERSION, () -> 7L); - Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, - ControllerGauge.VERSION.getGaugeName() + "." + compositeKey), 7); - - controllerMetrics.removePartitionGaugeForStreamTopic(tableWithType, topic, partitionGroupId, - ControllerGauge.VERSION); - Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); - } - - @Test - public void testPartitionGaugeWithOptionalStreamTopicDelegates() { - ControllerMetrics controllerMetrics = buildTestMetrics(); - String tableWithType = "myTable_REALTIME"; - String topic = "events"; - int partitionGroupId = 3; - String compositeKey = tableWithType + "-" + topic + "-" + partitionGroupId; - - controllerMetrics.setOrUpdatePartitionGauge(tableWithType, partitionGroupId, topic, ControllerGauge.VERSION, - () -> 7L); - Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, - ControllerGauge.VERSION.getGaugeName() + "." + compositeKey), 7); - - controllerMetrics.removePartitionGauge(tableWithType, partitionGroupId, topic, ControllerGauge.VERSION); - Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); - } - - @Test - public void testPartitionGaugeWithBlankOptionalStreamTopicMatchesPartitionOnly() { - ControllerMetrics controllerMetrics = buildTestMetrics(); - String table = "test_table"; - int partitionId = 1024; - - controllerMetrics.setOrUpdatePartitionGauge(table, partitionId, null, ControllerGauge.VERSION, () -> 2L); - Assert.assertEquals(MetricValueUtils.getGaugeValue(controllerMetrics, - ControllerGauge.VERSION.getGaugeName() + "." + table + "." + partitionId), 2); - - controllerMetrics.removePartitionGauge(table, partitionId, "", ControllerGauge.VERSION); - Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty()); - } - @Test public void testAddCallbackGauges() { ControllerMetrics controllerMetrics = buildTestMetrics(); From 4498c4e5cc5669581dd68e7a88a55ba8964942c3 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 19:59:07 +0530 Subject: [PATCH 40/53] Update RealtimeTableDataManager.java --- .../manager/realtime/RealtimeTableDataManager.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index ccb16a6e2b10..9a69e32d2d93 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -325,19 +325,16 @@ protected void doShutdown() { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing * the segment name) - * @param streamTopicName stream topic for this consumer (e.g. Kafka topic); used for ingestion - * delay metric labels * @param ingestionTimeMs ingestion time of the last consumed message (from * {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) * @param currentOffset offset of the last consumed message (from {@link StreamMessageMetadata}) */ - public void updateIngestionMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, - long ingestionTimeMs, long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { - _ingestionDelayTracker.updateMetrics(segmentName, partitionId, streamTopicName, ingestionTimeMs, - firstStreamIngestionTimeMs, currentOffset); - } + public void updateIngestionMetrics(String segmentName, int partitionId, long ingestionTimeMs, + long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { + _ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, firstStreamIngestionTimeMs, + currentOffset); /** * Returns the ingestion time of the last consumed message for the partition of the given segment. Returns From d0b3415bd4026c30749afa6357b519b4d13e4074 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:06:24 +0530 Subject: [PATCH 41/53] Update RealtimeSegmentDataManager.java --- .../realtime/RealtimeSegmentDataManager.java | 90 ++++++++----------- 1 file changed, 38 insertions(+), 52 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 823755dc3043..fa423c7d081c 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -315,7 +315,6 @@ public void deleteSegmentFile() { private final int _streamPartitionId; private final PartitionGroupConsumptionStatus _partitionGroupConsumptionStatus; final String _clientId; - private final String _streamConsumerMetricBaseKey; private final TransformPipeline _transformPipeline; private PartitionGroupConsumer _partitionGroupConsumer = null; private StreamMetadataProvider _partitionMetadataProvider = null; @@ -484,14 +483,14 @@ protected boolean consumeLoop() _segmentLogger.info("Starting consumption loop start offset {}, finalOffset {}", _currentOffset, _finalOffset); while (!_shouldStop && !endCriteriaReached()) { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 1); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 1); // Consume for the next readTime ms, or we get to final offset, whichever happens earlier, // Update _currentOffset upon return from this method MessageBatch messageBatch; try { messageBatch = _partitionGroupConsumer.fetchMessages(_currentOffset, _streamConfig.getFetchTimeoutMillis()); //track realtime rows fetched on a table level. This included valid + invalid rows - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_FETCHED, + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FETCHED, messageBatch.getUnfilteredMessageCount()); if (_segmentLogger.isDebugEnabled()) { _segmentLogger.debug("message batch received. filtered={} unfiltered={} endOfPartitionGroup={}", @@ -532,7 +531,7 @@ protected boolean consumeLoop() // TODO Issue 5359 Need to find a way to bump metrics without getting actual offset value. if (_currentOffset instanceof LongMsgOffset) { // TODO: only LongMsgOffset supplies long offset value. - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.HIGHEST_STREAM_OFFSET_CONSUMED, + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.HIGHEST_STREAM_OFFSET_CONSUMED, ((LongMsgOffset) _currentOffset).getOffset()); } lastUpdatedOffset = _streamPartitionMsgOffsetFactory.create(_currentOffset); @@ -573,12 +572,11 @@ protected boolean consumeLoop() } if (_numRowsErrored > 0) { - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); _serverMetrics.addMeteredTableValue(_tableStreamName, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored); // TODO Although the metric is called real-time, updating it at this point is not really real-time. The choice of // name is partly to avoid a more convoluted name and partly in anticipation of making it real-time. - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_BYTES_DROPPED, - _numBytesDropped); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_BYTES_DROPPED, _numBytesDropped); } return true; } @@ -659,8 +657,8 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } // Silently drop the row with error realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, - ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, realtimeRowsDroppedMeter); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1, + realtimeRowsDroppedMeter); _numRowsErrored++; _numBytesDropped += rowSizeInBytes; } @@ -679,7 +677,7 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee if (result != null) { if (result.getSkippedRowCount() > 0) { realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_FILTERED, + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, result.getSkippedRowCount(), realtimeRowsDroppedMeter); if (_trackFilteredMessageOffsets) { _filteredMessageOffsets.add(offset.toString()); @@ -687,13 +685,12 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee } if (result.getIncompleteRowCount() > 0) { realtimeIncompleteRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, - ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, result.getIncompleteRowCount(), - realtimeIncompleteRowsConsumedMeter); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.INCOMPLETE_REALTIME_ROWS_CONSUMED, + result.getIncompleteRowCount(), realtimeIncompleteRowsConsumedMeter); } if (result.getSanitizedRowCount() > 0) { realtimeRowsSanitizedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_SANITIZED, + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_SANITIZED, result.getSanitizedRowCount(), realtimeRowsSanitizedMeter); } List transformedRows = result.getTransformedRows(); @@ -704,15 +701,15 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee _lastRowMetadata = metadata; _lastConsumedTimestampMs = System.currentTimeMillis(); realtimeRowsConsumedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.REALTIME_ROWS_CONSUMED, - 1, realtimeRowsConsumedMeter); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_CONSUMED, 1, + realtimeRowsConsumedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_ROWS_CONSUMED, 1L); int recordSerializedValueLength = _lastRowMetadata.getRecordSerializedSize(); if (recordSerializedValueLength > 0) { realtimeBytesIngestedMeter = - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, - ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength, realtimeBytesIngestedMeter); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_BYTES_CONSUMED, + recordSerializedValueLength, realtimeBytesIngestedMeter); _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_BYTES_CONSUMED, recordSerializedValueLength); } } catch (Exception e) { @@ -836,21 +833,20 @@ public void run() { if (_state.shouldConsume()) { consumeLoop(); // Consume until we reached the end criteria, or we are stopped. } - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); if (_shouldStop) { break; } if (_state == State.INITIAL_CONSUMING) { initialConsumptionEnd = now(); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_INITIAL_CONSUMPTION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(initialConsumptionEnd - _startTimeMs)); } else if (_state == State.CATCHING_UP) { catchUpTimeMillis += now() - lastCatchUpStart; _serverMetrics - .setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, + .setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(catchUpTimeMillis)); } @@ -947,8 +943,7 @@ public void run() { // respectively. // Refer to the PR for the new commit protocol: https://github.com/apache/pinot/pull/14741 if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 1); if (!startSegmentCommit()) { // If for any reason commit failed, we don't want to be in COMMITTING state when we hold. // Change the state to HOLDING before looping around. @@ -958,8 +953,7 @@ public void run() { break; } } else { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED, 0); } long buildTimeSeconds = response.getBuildTimeSeconds(); try { @@ -1008,7 +1002,7 @@ public void run() { _segmentLogger.error(errorMessage, e); _state = State.ERROR; _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); postStopConsumedMsg(e.getClass().getName()); return; } @@ -1018,8 +1012,7 @@ public void run() { if (initialConsumptionEnd != 0L) { _serverMetrics - .setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, + .setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(now() - initialConsumptionEnd)); } // There is a race condition that the destroy() method can be called which ends up calling stop on the consumer. @@ -1029,7 +1022,7 @@ public void run() { // so it is ok not to mark it non-consuming, as the main thread will clean up this metric in destroy() method // as the final step. if (!_shouldStop) { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); } } @@ -1267,11 +1260,9 @@ protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) } long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_DURATION_SECONDS, TimeUnit.MILLISECONDS.toSeconds(buildTimeMillis)); - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, - ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, TimeUnit.MILLISECONDS.toSeconds(waitTimeMillis)); if (forCommit) { @@ -1322,7 +1313,7 @@ protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) private void reportSegmentBuildFailure(String errorMessage, @Nullable Exception e) { _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.SEGMENT_BUILD_FAILURE, 1); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.SEGMENT_BUILD_FAILURE, 1); } @VisibleForTesting @@ -1437,10 +1428,10 @@ private void releaseConsumerSemaphore() { * which no longer resides in this host any more, thus causes false positive information to the metric system. */ private void cleanupMetrics() { - _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING); - _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.STREAM_DATA_LOSS); - _serverMetrics.removeTableGauge(_streamConsumerMetricBaseKey, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED); - _serverMetrics.removeTableMeter(_streamConsumerMetricBaseKey, ServerMeter.SEGMENT_BUILD_FAILURE); + _serverMetrics.removeTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING); + _serverMetrics.removeTableGauge(_clientId, ServerGauge.STREAM_DATA_LOSS); + _serverMetrics.removeTableGauge(_clientId, ServerGauge.PAUSELESS_CONSUMPTION_ENABLED); + _serverMetrics.removeTableMeter(_clientId, ServerMeter.SEGMENT_BUILD_FAILURE); } protected void hold() @@ -1516,7 +1507,7 @@ private void removeSegmentFile() { public void goOnlineFromConsuming(SegmentZKMetadata segmentZKMetadata) throws InterruptedException { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); try { // Remove the segment file before we do anything else. removeSegmentFile(); @@ -1634,7 +1625,7 @@ public void goOnlineFromConsuming(SegmentZKMetadata segmentZKMetadata) } catch (Exception e) { Utils.rethrowException(e); } finally { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); } } @@ -1665,7 +1656,7 @@ private boolean catchupToFinalOffset(StreamPartitionMsgOffset endOffset, long ti _segmentLogger.warn("Exception when catching up to final offset", e); return false; } finally { - _serverMetrics.setValueOfTableGauge(_streamConsumerMetricBaseKey, ServerGauge.LLC_PARTITION_CONSUMING, 0); + _serverMetrics.setValueOfTableGauge(_clientId, ServerGauge.LLC_PARTITION_CONSUMING, 0); } if (_currentOffset.compareTo(endOffset) != 0) { // Timeout? @@ -1802,8 +1793,6 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf } else { _clientId = _tableNameWithType + "-" + streamTopic + "-" + _streamPartitionId; } - _streamConsumerMetricBaseKey = - IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix(_tableNameWithType, streamTopic, _streamPartitionId); _segmentLogger = LoggerFactory.getLogger(RealtimeSegmentDataManager.class.getName() + "_" + _segmentNameStr); _tableStreamName = _tableNameWithType + "_" + streamTopic; if (indexLoadingConfig.isRealtimeOffHeapAllocation() && !indexLoadingConfig.isDirectRealtimeOffHeapAllocation()) { @@ -2081,8 +2070,7 @@ private void makeStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to create stream consumer for topic partition {} reason {}", _clientId, reason, e); - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, - 1L); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); throw e; } } @@ -2103,8 +2091,7 @@ private void recreateStreamConsumer(String reason) { } catch (Exception e) { _segmentLogger.error("Faced exception while trying to recreate stream consumer for topic partition {}", _clientId, e); - _serverMetrics.addMeteredTableValue(_streamConsumerMetricBaseKey, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, - 1L); + _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.STREAM_CONSUMER_CREATE_EXCEPTIONS, 1L); throw e; } } @@ -2123,8 +2110,7 @@ private void updateIngestionMetrics(StreamMessageMetadata metadata) { if (metadata != null) { try { _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, - _streamConfig.getTopicName(), metadata.getRecordIngestionTimeMs(), - metadata.getFirstStreamRecordIngestionTimeMs(), metadata.getOffset()); + metadata.getRecordIngestionTimeMs(), metadata.getFirstStreamRecordIngestionTimeMs(), metadata.getOffset()); } catch (Exception e) { _segmentLogger.warn("Failed to update the ingestion metrics", e); } @@ -2137,8 +2123,8 @@ private void updateIngestionMetrics(StreamMessageMetadata metadata) { */ private void setIngestionDelayToZero() { long currentTimeMs = System.currentTimeMillis(); - _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, _streamConfig.getTopicName(), - currentTimeMs, currentTimeMs, null); + _realtimeTableDataManager.updateIngestionMetrics(_segmentNameStr, _partitionGroupId, currentTimeMs, currentTimeMs, + null); } // This should be done during commit? We may not always commit when we build a segment.... From 819f60bb7289d4de1099633c93ba3cf89341b85d Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:10:10 +0530 Subject: [PATCH 42/53] Update IngestionDelayTrackerTest.java --- .../realtime/IngestionDelayTrackerTest.java | 52 ++++++------------- 1 file changed, 17 insertions(+), 35 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java index c5f4e1ac79aa..80d5444e3ed9 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTrackerTest.java @@ -210,8 +210,7 @@ public void testRecordIngestionDelayWithNoAging() { // Test we follow a single partition up and down for (long ingestionTimeMs = 0; ingestionTimeMs <= maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, - firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -223,8 +222,7 @@ public void testRecordIngestionDelayWithNoAging() { // Test tracking down a measure for a given partition for (long ingestionTimeMs = maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, - firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), @@ -233,13 +231,12 @@ public void testRecordIngestionDelayWithNoAging() { } // Make the current partition maximum - ingestionDelayTracker.updateMetrics(segment0, partition0, null, maxTestDelay, maxTestDelay, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, maxTestDelay, maxTestDelay, null); // Bring up partition1 delay up and verify values for (long ingestionTimeMs = 0; ingestionTimeMs <= 2 * maxTestDelay; ingestionTimeMs++) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, - firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -250,8 +247,7 @@ public void testRecordIngestionDelayWithNoAging() { // Bring down values of partition1 and verify values for (long ingestionTimeMs = 2 * maxTestDelay; ingestionTimeMs >= 0; ingestionTimeMs--) { long firstStreamIngestionTimeMs = ingestionTimeMs + 1; - ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, - firstStreamIngestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, firstStreamIngestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), clock.millis() - ingestionTimeMs); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), @@ -284,7 +280,7 @@ public void testRecordIngestionDelayWithAging() { Clock clock = Clock.fixed(now, zoneId); ingestionDelayTracker.setClock(clock); long ingestionTimeMs = clock.millis() - partition0Delay0; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), partition0Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), partition0Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); @@ -299,7 +295,7 @@ public void testRecordIngestionDelayWithAging() { Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); ingestionTimeMs = offsetClock.millis() - partition0Delay1; - ingestionDelayTracker.updateMetrics(segment0, partition0, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment0, partition0, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition0), partition0Delay1); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition0), partition0Delay1); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition0), ingestionTimeMs); @@ -311,7 +307,7 @@ public void testRecordIngestionDelayWithAging() { (partition0Delay1 + partition0Offset1Ms)); ingestionTimeMs = offsetClock.millis() - partition1Delay0; - ingestionDelayTracker.updateMetrics(segment1, partition1, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segment1, partition1, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partition1), partition1Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partition1), partition1Delay0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partition1), ingestionTimeMs); @@ -341,7 +337,7 @@ public void testStopTrackingIngestionDelay() { for (int partitionId = 0; partitionId <= maxTestDelay; partitionId++) { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - partitionId; - ingestionDelayTracker.updateMetrics(segmentName, partitionId, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(partitionId), ingestionTimeMs); @@ -369,7 +365,7 @@ public void testStopTrackingIngestionDelayWithSegment() { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - 10; - ingestionDelayTracker.updateMetrics(segmentName, 0, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, 0, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(0), 10); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0), 10); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), ingestionTimeMs); @@ -380,7 +376,7 @@ public void testStopTrackingIngestionDelayWithSegment() { Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), Long.MIN_VALUE); // Should not update metrics for removed segment - ingestionDelayTracker.updateMetrics(segmentName, 0, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, 0, ingestionTimeMs, ingestionTimeMs, null); Assert.assertNull(ingestionDelayTracker.getPartitionIngestionDelayMs(0)); Assert.assertNull(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0)); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), Long.MIN_VALUE); @@ -401,7 +397,7 @@ public void testShutdown() { for (int partitionId = 0; partitionId <= maxTestDelay; partitionId++) { String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, 123).getSegmentName(); long ingestionTimeMs = clock.millis() - partitionId; - ingestionDelayTracker.updateMetrics(segmentName, partitionId, null, ingestionTimeMs, ingestionTimeMs, null); + ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, ingestionTimeMs, null); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(partitionId), partitionId); Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(partitionId), partitionId); } @@ -427,7 +423,7 @@ public void testRecordIngestionDelayOffset() { StreamPartitionMsgOffset msgOffset0 = new LongMsgOffset(50); StreamPartitionMsgOffset latestOffset0 = new LongMsgOffset(150); partitionMsgOffsetMap.put(partition0, latestOffset0); - ingestionDelayTracker.updateMetrics(segment0, partition0, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); + ingestionDelayTracker.updateMetrics(segment0, partition0, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition0), 100); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition0), 150); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition0), 50); @@ -438,7 +434,7 @@ public void testRecordIngestionDelayOffset() { StreamPartitionMsgOffset msgOffset1 = new LongMsgOffset(50); StreamPartitionMsgOffset latestOffset1 = new LongMsgOffset(150); partitionMsgOffsetMap.put(partition1, latestOffset1); - ingestionDelayTracker.updateMetrics(segment1, partition1, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset1); + ingestionDelayTracker.updateMetrics(segment1, partition1, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset1); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition1), 100); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition1), 150); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition1), 50); @@ -448,7 +444,7 @@ public void testRecordIngestionDelayOffset() { msgOffset0 = new LongMsgOffset(150); latestOffset0 = new LongMsgOffset(200); partitionMsgOffsetMap.put(partition0, latestOffset0); - ingestionDelayTracker.updateMetrics(segment0, partition0, null, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); + ingestionDelayTracker.updateMetrics(segment0, partition0, Long.MIN_VALUE, Long.MIN_VALUE, msgOffset0); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionOffsetLag(partition0), 50); Assert.assertEquals(ingestionDelayTracker.getLatestPartitionOffset(partition0), 200); Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionConsumingOffset(partition0), 150); @@ -507,8 +503,8 @@ public void testIngestionDelay() { ingestionDelayTracker._partitionsHostedByThisServer.put(partition0, true); ingestionDelayTracker._partitionsHostedByThisServer.put(partition1, true); - ingestionDelayTracker.updateMetrics(segment0, partition0, null, System.currentTimeMillis(), - System.currentTimeMillis(), new LongMsgOffset(50)); + ingestionDelayTracker.updateMetrics(segment0, partition0, System.currentTimeMillis(), System.currentTimeMillis(), + new LongMsgOffset(50)); ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); Map partitionIdVsLatestOffset = new HashMap<>(); @@ -535,20 +531,6 @@ public void testIngestionDelay() { ingestionDelayTracker.shutdown(); } - @Test - public void testStreamTopicRegistersTopicTaggedGauges() { - IngestionDelayTracker ingestionDelayTracker = createTracker(); - Instant now = Instant.now(); - ingestionDelayTracker.setClock(Clock.fixed(now, ZoneId.systemDefault())); - String segment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, 123).getSegmentName(); - long ts = now.toEpochMilli(); - ingestionDelayTracker.updateMetrics(segment, 0, "test_topic", ts, ts, null); - Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionTimeMs(0), ts); - Assert.assertEquals(ingestionDelayTracker.getPartitionIngestionDelayMs(0).longValue(), 0L); - Assert.assertEquals(ingestionDelayTracker.getPartitionEndToEndIngestionDelayMs(0).longValue(), 0L); - ingestionDelayTracker.shutdown(); - } - private void verifyMetrics(Map>> partitionToMetricToValues) { Assert.assertEquals(partitionToMetricToValues.size(), 2); verifyPartition0(partitionToMetricToValues.get(0)); From 239b2e5f90c2e22dcfe530be4652f7d5caaa7c51 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:12:07 +0530 Subject: [PATCH 43/53] Update RealtimeTableDataManager.java --- .../core/data/manager/realtime/RealtimeTableDataManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 9a69e32d2d93..15ec71fe50f4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -335,6 +335,7 @@ public void updateIngestionMetrics(String segmentName, int partitionId, long ing long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { _ingestionDelayTracker.updateMetrics(segmentName, partitionId, ingestionTimeMs, firstStreamIngestionTimeMs, currentOffset); + } /** * Returns the ingestion time of the last consumed message for the partition of the given segment. Returns From fcb21a688d38c41c8e235fdb2b68775e03963706 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:12:41 +0530 Subject: [PATCH 44/53] Update AbstractMetrics.java --- .../java/org/apache/pinot/common/metrics/AbstractMetrics.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java index 05f4321c2c30..c594188855c9 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java @@ -751,6 +751,7 @@ public void removeTableGauge(final String tableName, final G gauge) { removeGauge(fullGaugeName); } + /** * Removes a table gauge given the table name, the partition id and the gauge. * The add/remove is expected to work correctly in case of being invoked across multiple threads. From bd3cb8ece925b04a8e33fe67a9c33a5cac176ed3 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:21:07 +0530 Subject: [PATCH 45/53] Update IngestionDelayTracker.java --- .../realtime/IngestionDelayTracker.java | 83 +++++++++++-------- 1 file changed, 48 insertions(+), 35 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index a56701fa385e..61f4e1171caa 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -35,11 +35,14 @@ import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.metrics.ServerGauge; import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.metrics.ServerTimer; import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConsumerFactory; @@ -53,7 +56,8 @@ import org.slf4j.LoggerFactory; /** - * A Class to track realtime ingestion delay for table partitions on a given server. + * Tracks realtime ingestion delay for table partitions on the current server. + * Ingestion gauges are keyed like stream consumer metrics ({@code tableNameWithType-topic-streamPartition}). * Highlights: * 1-An object of this class is hosted by each RealtimeTableDataManager. * 2-The object tracks ingestion delays for all partitions hosted by the current server for the given Realtime table. @@ -98,15 +102,12 @@ private static class IngestionInfo { @Nullable volatile StreamPartitionMsgOffset _currentOffset; volatile long _firstStreamIngestionTimeMs; - @Nullable - final String _streamTopicName; IngestionInfo(long ingestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset, - long firstStreamIngestionTimeMs, @Nullable String streamTopicName) { + long firstStreamIngestionTimeMs) { _ingestionTimeMs = ingestionTimeMs; _currentOffset = currentOffset; _firstStreamIngestionTimeMs = firstStreamIngestionTimeMs; - _streamTopicName = streamTopicName; } void update(long ingestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset, @@ -135,6 +136,8 @@ void update(long ingestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffs private final ServerMetrics _serverMetrics; private final String _tableNameWithType; private final String _metricName; + @Nullable + private final String _consumerClientIdSuffix; private final RealtimeTableDataManager _realTimeTableDataManager; private final BooleanSupplier _isServerReadyToServeQueries; private final Cache _segmentsToIgnore = @@ -179,6 +182,10 @@ public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithTy _serverMetrics = serverMetrics; _tableNameWithType = tableNameWithType; _metricName = tableNameWithType; + InstanceDataManagerConfig instanceDataManagerConfig = realtimeTableDataManager.getInstanceDataManagerConfig(); + String consumerClientIdSuffix = + instanceDataManagerConfig != null ? instanceDataManagerConfig.getConsumerClientIdSuffix() : null; + _consumerClientIdSuffix = StringUtils.isNotBlank(consumerClientIdSuffix) ? consumerClientIdSuffix : null; _realTimeTableDataManager = realtimeTableDataManager; _isServerReadyToServeQueries = isServerReadyToServeQueries; @@ -336,9 +343,9 @@ public Thread newThread(Runnable r) { */ private void removePartitionId(int partitionId) { _partitionsHostedByThisServer.remove(partitionId); - IngestionInfo removed = _ingestionInfoMap.remove(partitionId); + _ingestionInfoMap.remove(partitionId); _partitionsTracked.computeIfPresent(partitionId, (k, v) -> { - removeMetrics(partitionId, removed != null ? removed._streamTopicName : null); + removeMetrics(partitionId); return null; }); @@ -372,50 +379,58 @@ void setClock(Clock clock) { _clock = clock; } + /** Table key for ingestion gauges for {@code pinotPartitionId}, aligned with stream consumer metrics. */ + private String getIngestionGaugeTableKey(int pinotPartitionId) { + TableConfig tableConfig = _realTimeTableDataManager.getCachedTableConfigAndSchema().getLeft(); + List streamConfigs = IngestionConfigUtils.getStreamConfigs(tableConfig); + StreamConfig streamConfig = IngestionConfigUtils.getStreamConfigFromPinotPartitionId(streamConfigs, + pinotPartitionId); + int streamPartitionId = + IngestionConfigUtils.getStreamPartitionIdFromPinotPartitionId(tableConfig, pinotPartitionId); + return IngestionConfigUtils.getStreamIngestionMetricTableKey(_tableNameWithType, streamConfig.getTopicName(), + streamPartitionId, _consumerClientIdSuffix); + } + @VisibleForTesting void createMetrics(int partitionId) { int streamConfigIndex = IngestionConfigUtils.getStreamConfigIndexFromPinotPartitionId(partitionId); StreamMetadataProvider streamMetadataProvider = _streamConfigIndexToStreamMetadataProvider.get(streamConfigIndex); + String tableKey = getIngestionGaugeTableKey(partitionId); if (streamMetadataProvider != null && streamMetadataProvider.supportsOffsetLag()) { - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_OFFSET_LAG, + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_OFFSET_LAG, () -> getPartitionIngestionOffsetLag(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, - ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET, () -> getPartitionIngestionConsumingOffset(partitionId)); + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET, + () -> getPartitionIngestionConsumingOffset(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, - ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, () -> getLatestPartitionOffset(partitionId)); + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, + () -> getLatestPartitionOffset(partitionId)); } - IngestionInfo ingestionInfo = _ingestionInfoMap.get(partitionId); - @Nullable String streamTopicName = ingestionInfo != null ? ingestionInfo._streamTopicName : null; - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, streamTopicName, - ServerGauge.REALTIME_INGESTION_DELAY_MS, () -> getPartitionIngestionDelayMs(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, streamTopicName, - ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, () -> getPartitionEndToEndIngestionDelayMs(partitionId)); - _serverMetrics.setOrUpdatePartitionGauge(_metricName, partitionId, - ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS, () -> getPartitionIngestionReportingStatus(partitionId)); + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_DELAY_MS, + () -> getPartitionIngestionDelayMs(partitionId)); + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, + () -> getPartitionEndToEndIngestionDelayMs(partitionId)); + _serverMetrics.setOrUpdateTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS, + () -> getPartitionIngestionReportingStatus(partitionId)); LOGGER.info("Successfully created ingestion metrics for partition id: {}", partitionId); } - private void removeMetrics(int partitionId, @Nullable String streamTopicName) { + private void removeMetrics(int partitionId) { int streamConfigIndex = IngestionConfigUtils.getStreamConfigIndexFromPinotPartitionId(partitionId); StreamMetadataProvider streamMetadataProvider = _streamConfigIndexToStreamMetadataProvider.get(streamConfigIndex); + String tableKey = getIngestionGaugeTableKey(partitionId); // Remove all metrics associated with this partition if (streamMetadataProvider != null && streamMetadataProvider.supportsOffsetLag()) { - _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_OFFSET_LAG); - _serverMetrics.removePartitionGauge(_metricName, partitionId, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET); - _serverMetrics.removePartitionGauge(_metricName, partitionId, - ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_OFFSET_LAG); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); } - _serverMetrics.removePartitionGauge(_metricName, partitionId, streamTopicName, - ServerGauge.REALTIME_INGESTION_DELAY_MS); - _serverMetrics.removePartitionGauge(_metricName, partitionId, streamTopicName, - ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); - _serverMetrics.removePartitionGauge(_metricName, partitionId, - ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_DELAY_MS); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS); + _serverMetrics.removeTableGauge(tableKey, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS); LOGGER.info("Successfully removed ingestion metrics for partition id: {}", partitionId); } @@ -426,14 +441,12 @@ private void removeMetrics(int partitionId, @Nullable String streamTopicName) { * @param segmentName name of the consuming segment * @param partitionId partition id of the consuming segment (directly passed in to avoid parsing the * segment name) - * @param streamTopicName stream topic name for this consumer (e.g. Kafka topic); when blank, metrics use - * the legacy name without a topic label * @param ingestionTimeMs ingestion time of the last consumed message (from {@link StreamMessageMetadata}) * @param firstStreamIngestionTimeMs ingestion time of the last consumed message in the first stream (from * {@link StreamMessageMetadata}) * @param currentOffset offset of the last consumed message (from {@link StreamMessageMetadata}) */ - public void updateMetrics(String segmentName, int partitionId, @Nullable String streamTopicName, long ingestionTimeMs, + public void updateMetrics(String segmentName, int partitionId, long ingestionTimeMs, long firstStreamIngestionTimeMs, @Nullable StreamPartitionMsgOffset currentOffset) { if (!_isServerReadyToServeQueries.getAsBoolean() || _realTimeTableDataManager.isShutDown()) { // Do not update the ingestion delay metrics during server startup period @@ -452,7 +465,7 @@ public void updateMetrics(String segmentName, int partitionId, @Nullable String return v; } if (v == null) { - return new IngestionInfo(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs, streamTopicName); + return new IngestionInfo(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs); } v.update(ingestionTimeMs, currentOffset, firstStreamIngestionTimeMs); return v; From 278d959def18e3e34eaeee6e32412a00f66859b8 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:23:09 +0530 Subject: [PATCH 46/53] Update IngestionConfigUtils.java --- .../pinot/spi/utils/IngestionConfigUtils.java | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java index bd99dfd5fa43..b2dbd06fc303 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java @@ -141,20 +141,6 @@ public static int getStreamConfigIndexFromPinotPartitionId(int partitionId) { return partitionId / PARTITION_PADDING_OFFSET; } - /** - * Returns the stream consumer client id without the optional instance suffix from - * {@link org.apache.pinot.spi.config.instance.InstanceDataManagerConfig#getConsumerClientIdSuffix()}. - * Format: {@code tableNameWithType + "-" + streamTopic + "-" + partitionId}. - *

Use the stream topic partition id for {@code partitionId} when matching realtime consumer client ids. Other - * callers (for example stream-aware ingestion gauges) may pass the Pinot partition group id as the third segment; - * the string format matches server metrics keys that {@code server.yml} maps to Prometheus series with - * {@code topic} and {@code partition} labels. - */ - public static String getStreamConsumerClientIdWithoutSuffix(String tableNameWithType, String streamTopic, - int partitionId) { - return tableNameWithType + "-" + streamTopic + "-" + partitionId; - } - /** * Fetches the streamConfig from the list of streamConfigs according to the partition id. */ @@ -353,6 +339,23 @@ public static String getTableTopicUniqueClientId(String className, StreamConfig className + "-" + streamConfig.getTableNameWithType() + "-" + streamConfig.getTopicName()); } + /** + * Returns the table-key string for realtime stream server metrics: {@code table-topic-partition} and optional + * {@code consumerClientIdSuffix} when non-blank. + * + * @param tableNameWithType table name with type (e.g. {@code myTable_REALTIME}) + * @param topicName stream topic name + * @param streamPartitionId stream partition id + * @param consumerClientIdSuffix optional suffix; ignored if null or blank + */ + public static String getStreamIngestionMetricTableKey(String tableNameWithType, String topicName, + int streamPartitionId, String consumerClientIdSuffix) { + if (StringUtils.isNotBlank(consumerClientIdSuffix)) { + return tableNameWithType + "-" + topicName + "-" + streamPartitionId + "-" + consumerClientIdSuffix; + } + return tableNameWithType + "-" + topicName + "-" + streamPartitionId; + } + /** * Returns a Map of stream config index to Set of stream partition Ids. * @param pinotPartitionIds Set of pinot partition ids. From 0fa2e7f3ef72a189a6b2728feaefe70561b16042 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:23:56 +0530 Subject: [PATCH 47/53] Update IngestionConfigUtilsTest.java --- .../pinot/spi/utils/IngestionConfigUtilsTest.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java index c306a140b1b0..f3961e9fdf91 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java @@ -184,8 +184,15 @@ public void testGetStreamConfigIndexToStreamPartitions() { } @Test - public void testGetStreamConsumerClientIdWithoutSuffix() { - Assert.assertEquals(IngestionConfigUtils.getStreamConsumerClientIdWithoutSuffix( - "db.myTable_REALTIME", "events", 3), "db.myTable_REALTIME-events-3"); + public void testGetStreamIngestionMetricTableKey() { + Map streamConfigMap = new HashMap<>(); + streamConfigMap.put("streamType", "kafka"); + streamConfigMap.put("stream.kafka.topic.name", "events"); + StreamConfig streamConfig = new StreamConfig("db.myTable_REALTIME", streamConfigMap); + String tableWithType = "db.myTable_REALTIME"; + Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, + streamConfig.getTopicName(), 7, null), "db.myTable_REALTIME-events-7"); + Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, + streamConfig.getTopicName(), 7, "host1"), "db.myTable_REALTIME-events-7-host1"); } } From 622e6fd43bc6ef1815be1113d27e71adf085d5fe Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 20:26:06 +0530 Subject: [PATCH 48/53] Update ServerPrometheusMetricsTest.java --- .../metrics/prometheus/ServerPrometheusMetricsTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java index 79fb88f56ed7..07a3c3621ba8 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/ServerPrometheusMetricsTest.java @@ -50,13 +50,14 @@ public abstract class ServerPrometheusMetricsTest extends PinotPrometheusMetrics ServerGauge.LAST_REALTIME_SEGMENT_CREATION_WAIT_TIME_SECONDS, ServerGauge.LAST_REALTIME_SEGMENT_INITIAL_CONSUMPTION_DURATION_SECONDS, ServerGauge.LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS, - ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS); + ServerGauge.LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS, + ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.REALTIME_INGESTION_DELAY_MS, + ServerGauge.END_TO_END_REALTIME_INGESTION_DELAY_MS, ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, + ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET, ServerGauge.REALTIME_INGESTION_DELAY_REPORTING_STATUS); private static final List GAUGES_ACCEPTING_PARTITION = List.of(ServerGauge.UPSERT_VALID_DOC_ID_SNAPSHOT_COUNT, ServerGauge.UPSERT_PRIMARY_KEYS_IN_SNAPSHOT_COUNT, - ServerGauge.REALTIME_INGESTION_OFFSET_LAG, ServerGauge.REALTIME_INGESTION_DELAY_MS, - ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT, - ServerGauge.REALTIME_INGESTION_UPSTREAM_OFFSET, ServerGauge.REALTIME_INGESTION_CONSUMING_OFFSET); + ServerGauge.UPSERT_PRIMARY_KEYS_COUNT, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT); private static final List GAUGES_ACCEPTING_RAW_TABLE_NAME = List.of(ServerGauge.REALTIME_OFFHEAP_MEMORY_USED, ServerGauge.REALTIME_SEGMENT_NUM_PARTITIONS, From 12c911ba3d0c3c1ca81b368289874262519c77f0 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Tue, 19 May 2026 22:44:41 +0530 Subject: [PATCH 49/53] Update IngestionConfigUtilsTest.java --- .../org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java index f3961e9fdf91..09a9dd998592 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java @@ -32,6 +32,7 @@ import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.Assert; import org.testng.annotations.Test; From 90d7941c8f097a56e9b4c41318580de2f09342b8 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Wed, 20 May 2026 00:12:58 +0530 Subject: [PATCH 50/53] Update pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java Co-authored-by: Xiaotian (Jackie) Jiang <17555551+Jackie-Jiang@users.noreply.github.com> --- .../java/org/apache/pinot/spi/utils/IngestionConfigUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java index b2dbd06fc303..0ea93d50fdd3 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java @@ -349,7 +349,7 @@ public static String getTableTopicUniqueClientId(String className, StreamConfig * @param consumerClientIdSuffix optional suffix; ignored if null or blank */ public static String getStreamIngestionMetricTableKey(String tableNameWithType, String topicName, - int streamPartitionId, String consumerClientIdSuffix) { + int streamPartitionId, @Nullable String consumerClientIdSuffix) { if (StringUtils.isNotBlank(consumerClientIdSuffix)) { return tableNameWithType + "-" + topicName + "-" + streamPartitionId + "-" + consumerClientIdSuffix; } From 90663157903516c32d52aa60bec298ba2a119082 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Wed, 20 May 2026 00:19:55 +0530 Subject: [PATCH 51/53] Update IngestionConfigUtils.java --- .../java/org/apache/pinot/spi/utils/IngestionConfigUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java index 0ea93d50fdd3..1d7e736c43d2 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.config.table.IndexingConfig; From 8f12b1482d9f8ef5ce58a91753eec79df16b9315 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Wed, 20 May 2026 00:21:45 +0530 Subject: [PATCH 52/53] Update IngestionDelayTracker.java --- .../core/data/manager/realtime/IngestionDelayTracker.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java index 61f4e1171caa..520ae5cbaff1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java @@ -35,7 +35,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import javax.annotation.Nullable; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.metrics.ServerGauge; import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; @@ -183,9 +182,8 @@ public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithTy _tableNameWithType = tableNameWithType; _metricName = tableNameWithType; InstanceDataManagerConfig instanceDataManagerConfig = realtimeTableDataManager.getInstanceDataManagerConfig(); - String consumerClientIdSuffix = + _consumerClientIdSuffix = instanceDataManagerConfig != null ? instanceDataManagerConfig.getConsumerClientIdSuffix() : null; - _consumerClientIdSuffix = StringUtils.isNotBlank(consumerClientIdSuffix) ? consumerClientIdSuffix : null; _realTimeTableDataManager = realtimeTableDataManager; _isServerReadyToServeQueries = isServerReadyToServeQueries; From 0743774069d63eb7a5075a6b9ec94602fd25caa5 Mon Sep 17 00:00:00 2001 From: Radhika Patwari <45709641+rsrkpatwari1234@users.noreply.github.com> Date: Wed, 20 May 2026 01:50:03 +0530 Subject: [PATCH 53/53] Update IngestionConfigUtilsTest.java --- .../pinot/spi/utils/IngestionConfigUtilsTest.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java index 09a9dd998592..f8adb800c499 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/IngestionConfigUtilsTest.java @@ -32,7 +32,6 @@ import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.Assert; import org.testng.annotations.Test; @@ -186,14 +185,10 @@ public void testGetStreamConfigIndexToStreamPartitions() { @Test public void testGetStreamIngestionMetricTableKey() { - Map streamConfigMap = new HashMap<>(); - streamConfigMap.put("streamType", "kafka"); - streamConfigMap.put("stream.kafka.topic.name", "events"); - StreamConfig streamConfig = new StreamConfig("db.myTable_REALTIME", streamConfigMap); String tableWithType = "db.myTable_REALTIME"; - Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, - streamConfig.getTopicName(), 7, null), "db.myTable_REALTIME-events-7"); - Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, - streamConfig.getTopicName(), 7, "host1"), "db.myTable_REALTIME-events-7-host1"); + Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, "events", 7, null), + "db.myTable_REALTIME-events-7"); + Assert.assertEquals(IngestionConfigUtils.getStreamIngestionMetricTableKey(tableWithType, "events", 7, "host1"), + "db.myTable_REALTIME-events-7-host1"); } }