-
Notifications
You must be signed in to change notification settings - Fork 8
feat(metadata:inkless): Log metrics for Inkless #511
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
| @@ -0,0 +1,162 @@ | ||||
| /* | ||||
| * Inkless | ||||
| * Copyright (C) 2024 - 2025 Aiven OY | ||||
| * | ||||
| * This program is free software: you can redistribute it and/or modify | ||||
| * it under the terms of the GNU Affero General Public License as published by | ||||
| * the Free Software Foundation, either version 3 of the License, or | ||||
| * (at your option) any later version. | ||||
| * | ||||
| * This program is distributed in the hope that it will be useful, | ||||
| * but WITHOUT ANY WARRANTY; without even the implied warranty of | ||||
| * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||||
| * GNU Affero General Public License for the details. | ||||
| * | ||||
| * You should have received a copy of the GNU Affero General Public License | ||||
| * along with this program. If not, see <http://www.gnu.org/licenses/>. | ||||
| */ | ||||
| package io.aiven.inkless.metrics; | ||||
|
|
||||
| import org.apache.kafka.common.TopicIdPartition; | ||||
| import org.apache.kafka.common.protocol.Errors; | ||||
| import org.apache.kafka.server.metrics.KafkaMetricsGroup; | ||||
| import org.apache.kafka.storage.internals.log.LogMetricNames; | ||||
|
|
||||
| import java.io.Closeable; | ||||
| import java.io.IOException; | ||||
| import java.util.ArrayList; | ||||
| import java.util.LinkedHashMap; | ||||
| import java.util.List; | ||||
| import java.util.Map; | ||||
| import java.util.Set; | ||||
| import java.util.concurrent.ConcurrentHashMap; | ||||
|
|
||||
| import io.aiven.inkless.common.SharedState; | ||||
| import io.aiven.inkless.control_plane.ControlPlane; | ||||
| import io.aiven.inkless.control_plane.GetLogInfoRequest; | ||||
| import io.aiven.inkless.control_plane.GetLogInfoResponse; | ||||
| import io.aiven.inkless.control_plane.MetadataView; | ||||
|
|
||||
| /** | ||||
| * Exposes JMX metrics for diskless (Inkless) partitions that mirror the classic | ||||
| * {@code kafka.log:type=Log} metrics (LogStartOffset, LogEndOffset, Size). Values are | ||||
| * sourced from the Control Plane and tagged with topic and partition so existing | ||||
| * tooling and dashboards work unchanged. | ||||
| */ | ||||
| public final class InklessLogMetrics implements Runnable, Closeable { | ||||
|
|
||||
| private static final String METRICS_GROUP_PKG = "kafka.log"; | ||||
| private static final String METRICS_GROUP_TYPE = "Log"; | ||||
|
|
||||
| private final ControlPlane controlPlane; | ||||
| private final MetadataView metadataView; | ||||
| private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup(METRICS_GROUP_PKG, METRICS_GROUP_TYPE); | ||||
|
|
||||
| /** Cache of log info per partition, updated by {@link #run()}. */ | ||||
| private final ConcurrentHashMap<TopicIdPartition, LogInfoSnapshot> cache = new ConcurrentHashMap<>(); | ||||
|
|
||||
| /** Partitions that currently have gauges registered, for diffing on refresh. */ | ||||
| private final Set<TopicIdPartition> registeredPartitions = ConcurrentHashMap.newKeySet(); | ||||
|
|
||||
| public InklessLogMetrics(final SharedState sharedState) { | ||||
| this(sharedState.controlPlane(), sharedState.metadata()); | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Constructor for tests; avoids building a full SharedState. | ||||
| */ | ||||
| InklessLogMetrics(final ControlPlane controlPlane, final MetadataView metadataView) { | ||||
| this.controlPlane = controlPlane; | ||||
| this.metadataView = metadataView; | ||||
| run(); | ||||
|
||||
| run(); |
Copilot
AI
Feb 16, 2026
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
getLogInfo responses are consumed positionally (responses.get(i)), but ControlPlane#getLogInfo does not document (or enforce) an in-order response contract the way findBatches does. The Postgres implementation builds a VALUES table without an explicit ordering, which can legally return rows in a different order, leading to metrics being attributed to the wrong partition. Consider changing the API to return keyed results (include topicId/partition in the response or return a Map<TopicIdPartition, ...>), or add an explicit request index and guarantee ordering end-to-end; at minimum, validate responses.size() before indexing.
Copilot
AI
Feb 16, 2026
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LogMetricNames.LOG_END_OFFSET in Kafka’s UnifiedLog is logEndOffset (LEO), but this gauge is currently wired to highWatermark. If Inkless only has a high watermark available, exposing it under the LogEndOffset name will make existing dashboards/alerts misleading. Either source/report the actual log end offset from the Control Plane, or expose high watermark under a distinct metric name/tag so semantics stay consistent with classic kafka.log:type=Log metrics.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,152 @@ | ||
| /* | ||
| * Inkless | ||
| * Copyright (C) 2024 - 2025 Aiven OY | ||
| * | ||
| * This program is free software: you can redistribute it and/or modify | ||
| * it under the terms of the GNU Affero General Public License as published by | ||
| * the Free Software Foundation, either version 3 of the License, or | ||
| * (at your option) any later version. | ||
| * | ||
| * This program is distributed in the hope that it will be useful, | ||
| * but WITHOUT ANY WARRANTY; without even the implied warranty of | ||
| * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||
| * GNU Affero General Public License for the details. | ||
| * | ||
| * You should have received a copy of the GNU Affero General Public License | ||
| * along with this program. If not, see <http://www.gnu.org/licenses/>. | ||
| */ | ||
| package io.aiven.inkless.metrics; | ||
|
|
||
| import org.apache.kafka.common.TopicIdPartition; | ||
| import org.apache.kafka.common.Uuid; | ||
| import org.apache.kafka.server.metrics.KafkaYammerMetrics; | ||
| import org.apache.kafka.storage.internals.log.LogMetricNames; | ||
|
|
||
| import com.yammer.metrics.core.Gauge; | ||
| import com.yammer.metrics.core.MetricName; | ||
|
|
||
| import org.junit.jupiter.api.AfterEach; | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.extension.ExtendWith; | ||
| import org.mockito.Mock; | ||
| import org.mockito.junit.jupiter.MockitoExtension; | ||
| import org.mockito.junit.jupiter.MockitoSettings; | ||
| import org.mockito.quality.Strictness; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Set; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import io.aiven.inkless.control_plane.ControlPlane; | ||
| import io.aiven.inkless.control_plane.GetLogInfoResponse; | ||
| import io.aiven.inkless.control_plane.MetadataView; | ||
|
|
||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.mockito.ArgumentMatchers.anyList; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| @ExtendWith(MockitoExtension.class) | ||
| @MockitoSettings(strictness = Strictness.STRICT_STUBS) | ||
| class InklessLogMetricsTest { | ||
|
|
||
| private static final String TOPIC = "diskless-metrics-test-topic"; | ||
| private static final Uuid TOPIC_ID = new Uuid(100L, 200L); | ||
| private static final TopicIdPartition T0P0 = new TopicIdPartition(TOPIC_ID, 0, TOPIC); | ||
| private static final TopicIdPartition T0P1 = new TopicIdPartition(TOPIC_ID, 1, TOPIC); | ||
|
|
||
| @Mock | ||
| private ControlPlane controlPlane; | ||
| @Mock | ||
| private MetadataView metadataView; | ||
|
|
||
| private InklessLogMetrics metrics; | ||
|
|
||
| @BeforeEach | ||
| void setUp() { | ||
| when(metadataView.getDisklessTopicPartitions()).thenReturn(Set.of()); | ||
| } | ||
|
|
||
| @AfterEach | ||
| void tearDown() { | ||
| if (metrics != null) { | ||
| try { | ||
| metrics.close(); | ||
| } catch (Exception ignored) { | ||
| // cleanup | ||
| } | ||
| } | ||
| removeDisklessLogMetricsFromRegistry(); | ||
| } | ||
|
|
||
| private static void removeDisklessLogMetricsFromRegistry() { | ||
| List<MetricName> toRemove = KafkaYammerMetrics.defaultRegistry().allMetrics().keySet().stream() | ||
| .filter(n -> "kafka.log".equals(n.getGroup()) && "Log".equals(n.getType()) | ||
| && (LogMetricNames.LOG_START_OFFSET.equals(n.getName()) | ||
| || LogMetricNames.LOG_END_OFFSET.equals(n.getName()) | ||
| || LogMetricNames.SIZE.equals(n.getName()))) | ||
| .filter(n -> n.getMBeanName().contains("topic=" + TOPIC)) | ||
| .collect(Collectors.toList()); | ||
| toRemove.forEach(KafkaYammerMetrics.defaultRegistry()::removeMetric); | ||
| } | ||
|
|
||
| private static long gaugeValue(String metricName, String topic, int partition) { | ||
| String suffix = ",name=" + metricName + ",topic=" + topic + ",partition=" + partition; | ||
| return KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream() | ||
| .filter(e -> e.getKey().getMBeanName().endsWith(suffix)) | ||
| .findFirst() | ||
| .map(e -> ((Number) ((Gauge<?>) e.getValue()).value()).longValue()) | ||
| .orElse(-999L); | ||
| } | ||
|
|
||
| @Test | ||
| void registersGaugesAndReturnsValuesFromGetLogInfo() { | ||
| when(metadataView.getDisklessTopicPartitions()).thenReturn(Set.of(T0P0, T0P1)); | ||
| when(controlPlane.getLogInfo(anyList())).thenReturn(List.of( | ||
| GetLogInfoResponse.success(0L, 10L, 100L), | ||
| GetLogInfoResponse.success(1L, 11L, 200L) | ||
| )); | ||
|
|
||
| metrics = new InklessLogMetrics(controlPlane, metadataView); | ||
|
|
||
| assertThat(gaugeValue(LogMetricNames.LOG_START_OFFSET, TOPIC, 0)).isEqualTo(0L); | ||
| assertThat(gaugeValue(LogMetricNames.LOG_END_OFFSET, TOPIC, 0)).isEqualTo(10L); | ||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 0)).isEqualTo(100L); | ||
| assertThat(gaugeValue(LogMetricNames.LOG_START_OFFSET, TOPIC, 1)).isEqualTo(1L); | ||
| assertThat(gaugeValue(LogMetricNames.LOG_END_OFFSET, TOPIC, 1)).isEqualTo(11L); | ||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 1)).isEqualTo(200L); | ||
| } | ||
|
|
||
| @Test | ||
| void removesGaugesWhenPartitionDisappears() { | ||
| when(metadataView.getDisklessTopicPartitions()).thenReturn(Set.of(T0P0, T0P1)); | ||
| when(controlPlane.getLogInfo(anyList())).thenReturn(List.of( | ||
| GetLogInfoResponse.success(0L, 10L, 100L), | ||
| GetLogInfoResponse.success(1L, 11L, 200L) | ||
| )); | ||
|
|
||
| metrics = new InklessLogMetrics(controlPlane, metadataView); | ||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 1)).isEqualTo(200L); | ||
|
|
||
| when(metadataView.getDisklessTopicPartitions()).thenReturn(Set.of(T0P0)); | ||
| when(controlPlane.getLogInfo(anyList())).thenReturn(List.of(GetLogInfoResponse.success(0L, 10L, 100L))); | ||
| metrics.run(); | ||
|
|
||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 0)).isEqualTo(100L); | ||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 1)).isEqualTo(-999L); | ||
| } | ||
|
|
||
| @Test | ||
| void closeRemovesAllGauges() throws Exception { | ||
| when(metadataView.getDisklessTopicPartitions()).thenReturn(Set.of(T0P0)); | ||
| when(controlPlane.getLogInfo(anyList())).thenReturn(List.of(GetLogInfoResponse.success(0L, 5L, 50L))); | ||
|
|
||
| metrics = new InklessLogMetrics(controlPlane, metadataView); | ||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 0)).isEqualTo(50L); | ||
|
|
||
| metrics.close(); | ||
| metrics = null; | ||
|
|
||
| assertThat(gaugeValue(LogMetricNames.SIZE, TOPIC, 0)).isEqualTo(-999L); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
InklessLogMetricsis refreshed via a hard-coded30000Linterval. Since other Inkless background tasks take their intervals from config (fileCleanerInterval,fileMergerInterval), consider making this refresh interval configurable (or at least define it as a named constant/Duration) to avoid magic numbers and to allow tuning in production if the Control Plane load or desired freshness changes.