Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -45,9 +45,7 @@ docs:

.PHONY: fmt
fmt:
./gradlew :core:spotlessJavaApply
./gradlew :metadata:spotlessJavaApply
./gradlew :storage:inkless:spotlessJavaApply
./gradlew :core:spotlessJavaApply :metadata:spotlessJavaApply :storage:spotlessJavaApply :storage:inkless:spotlessJavaApply

.PHONY: test
test:
Expand Down
14 changes: 4 additions & 10 deletions core/src/main/scala/kafka/server/KafkaApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -679,11 +679,8 @@ class KafkaApis(val requestChannel: RequestChannel,
// If the topic name was not known, we will have no bytes out.
if (topicResponse.topic != null) {
val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, data.partitionIndex))
if (inklessSharedState.exists(_.metadata().isDisklessTopic(tp.topic))) {
brokerTopicStats.updateBytesOutForDisklessTopic(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
} else {
brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
}
val isDiskless = inklessSharedState.exists(_.metadata().isDisklessTopic(tp.topic))
brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data), isDiskless)
}
}
}
Expand Down Expand Up @@ -4078,11 +4075,8 @@ class KafkaApis(val requestChannel: RequestChannel,
// If the topic name was not known, we will have no bytes out.
if (topicResponse.topicId != null) {
val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicIdNames.get(topicResponse.topicId), data.partitionIndex))
if (inklessSharedState.exists(_.metadata().isDisklessTopic(tp.topic))) {
brokerTopicStats.updateBytesOutForDisklessTopic(tp.topic, false, false, ShareFetchResponse.recordsSize(data))
} else {
brokerTopicStats.updateBytesOut(tp.topic, false, false, ShareFetchResponse.recordsSize(data))
}
val isDiskless = inklessSharedState.exists(_.metadata().isDisklessTopic(tp.topic))
brokerTopicStats.updateBytesOut(tp.topic, false, false, ShareFetchResponse.recordsSize(data), isDiskless)
}
}
}
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/kafka/server/ReplicaManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1479,8 +1479,8 @@ class ReplicaManager(val config: KafkaConfig,
val numAppendedMessages = info.numMessages

// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
brokerTopicStats.topicStats(topicIdPartition.topic).bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.allTopicsStats.bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.topicStats(topicIdPartition.topic).bytesInRate().mark(records.sizeInBytes)
brokerTopicStats.allTopicsStats.bytesInRate(false).mark(records.sizeInBytes)
Comment on lines +1482 to +1483
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To make this consistent, bytesInRate() in both.

brokerTopicStats.topicStats(topicIdPartition.topic).messagesInRate.mark(numAppendedMessages)
brokerTopicStats.allTopicsStats.messagesInRate.mark(numAppendedMessages)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,8 +135,8 @@ CompletableFuture<Map<TopicIdPartition, PartitionResponse>> add(
);

// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
brokerTopicStats.topicStats(topicIdPartition.topic()).bytesInRateDisklessTopicType().mark(records.sizeInBytes());
brokerTopicStats.allTopicsStats().bytesInRateDisklessTopicType().mark(records.sizeInBytes());
brokerTopicStats.topicStats(topicIdPartition.topic()).bytesInRate(true).mark(records.sizeInBytes());
brokerTopicStats.allTopicsStats().bytesInRate(true).mark(records.sizeInBytes());
brokerTopicStats.topicStats(topicIdPartition.topic()).messagesInRate().mark(appendInfo.numMessages());
brokerTopicStats.allTopicsStats().messagesInRate().mark(appendInfo.numMessages());
// case e@ (_: UnknownTopicOrPartitionException | // Handled earlier
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.yammer.metrics.core.Meter;

import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
Expand All @@ -32,9 +33,7 @@
public final class BrokerTopicMetrics {
public static final String MESSAGE_IN_PER_SEC = "MessagesInPerSec";
public static final String BYTES_IN_PER_SEC = "BytesInPerSec";
public static final String BYTES_IN_PER_SEC_DISKLESS_TOPIC = "BytesInPerSecDisklessTopic"; // The metric is BYTES_IN_PER_SEC
public static final String BYTES_OUT_PER_SEC = "BytesOutPerSec";
public static final String BYTES_OUT_PER_SEC_DISKLESS_TOPIC = "BytesOutPerSecDisklessTopic"; // The metric is BYTES_OUT_PER_SEC
public static final String BYTES_REJECTED_PER_SEC = "BytesRejectedPerSec";
public static final String REPLICATION_BYTES_IN_PER_SEC = "ReplicationBytesInPerSec";
public static final String REPLICATION_BYTES_OUT_PER_SEC = "ReplicationBytesOutPerSec";
Expand All @@ -60,10 +59,19 @@ public final class BrokerTopicMetrics {
// For backward compatibility, we keep the old package name as metric group name.
private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "BrokerTopicMetrics");
private final Map<String, String> tags;
private final Map<String, String> tagsForClassicTopic;
private final Map<String, String> tagsForDisklessTopic;
private final Map<String, MeterWrapper> metricTypeMap = new java.util.HashMap<>();
private final Map<String, GaugeWrapper> metricGaugeTypeMap = new java.util.HashMap<>();
// topicType tags are only used for broker-level "all topics" meters.
// They intentionally do NOT depend on any topic name: they represent a broker-wide split of traffic.
// We also intentionally avoid adding topicType tags to per-topic metrics to prevent metric cardinality
// explosion (classic + diskless variants for every topic would double the meters kept in memory).
private final Map<String, String> classicTags = Map.of("topicType", "classic");
private final Map<String, String> disklessTags = Map.of("topicType", "diskless");
Comment on lines +66 to +67
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These could even be

private static final Map<String, String> CLASSIC_TAGS = Map.of("topicType", "classic");
private static final Map<String, String> DISKLESS_TAGS = Map.of("topicType", "diskless");

private final boolean isAllTopicsStats;
private final Map<String, MeterWrapper> metricTypeMap = new HashMap<>();
private final Map<String, GaugeWrapper> metricGaugeTypeMap = new HashMap<>();
// Only used for all-topics bytes metrics (topicType=diskless). Classic meters are stored in metricTypeMap.
// These are not additional metric names, but mappings to existing bytesIn/Out names + diskless topic type tags.
private final MeterWrapper disklessBytesInRate;
private final MeterWrapper disklessBytesOutRate;

public BrokerTopicMetrics(boolean remoteStorageEnabled) {
this(Optional.empty(), remoteStorageEnabled);
Expand All @@ -75,14 +83,30 @@ public BrokerTopicMetrics(String name, boolean remoteStorageEnabled) {

private BrokerTopicMetrics(Optional<String> name, boolean remoteStorageEnabled) {
this.tags = name.map(s -> Map.of("topic", s)).orElse(Map.of());
this.tagsForClassicTopic = name.map(s -> Map.of("topic", s, "topicType", "classic")).orElse(Map.of("topicType", "classic"));
this.tagsForDisklessTopic = name.map(s -> Map.of("topic", s, "topicType", "diskless")).orElse(Map.of("topicType", "diskless"));
this.isAllTopicsStats = name.isEmpty();

metricTypeMap.put(MESSAGE_IN_PER_SEC, new MeterWrapper(MESSAGE_IN_PER_SEC, "messages"));
metricTypeMap.put(BYTES_IN_PER_SEC, new MeterWrapper(BYTES_IN_PER_SEC, "bytes", tagsForClassicTopic));
metricTypeMap.put(BYTES_IN_PER_SEC_DISKLESS_TOPIC, new MeterWrapper(BYTES_IN_PER_SEC, "bytes", tagsForDisklessTopic));
metricTypeMap.put(BYTES_OUT_PER_SEC, new MeterWrapper(BYTES_OUT_PER_SEC, "bytes", tagsForClassicTopic));
metricTypeMap.put(BYTES_OUT_PER_SEC_DISKLESS_TOPIC, new MeterWrapper(BYTES_OUT_PER_SEC, "bytes", tagsForDisklessTopic));

final MeterWrapper classicBytesInRate;
final MeterWrapper classicBytesOutRate;
if (isAllTopicsStats) {
classicBytesInRate = new MeterWrapper(BYTES_IN_PER_SEC, "bytes", classicTags);
classicBytesOutRate = new MeterWrapper(BYTES_OUT_PER_SEC, "bytes", classicTags);
disklessBytesInRate = new MeterWrapper(BYTES_IN_PER_SEC, "bytes", disklessTags);
disklessBytesOutRate = new MeterWrapper(BYTES_OUT_PER_SEC, "bytes", disklessTags);

// Keep broker-level behavior: eagerly initialize the classic meters.
classicBytesInRate.meter();
classicBytesOutRate.meter();
} else {
classicBytesInRate = new MeterWrapper(BYTES_IN_PER_SEC, "bytes", tags);
classicBytesOutRate = new MeterWrapper(BYTES_OUT_PER_SEC, "bytes", tags);
disklessBytesInRate = null;
disklessBytesOutRate = null;
}
metricTypeMap.put(BYTES_IN_PER_SEC, classicBytesInRate);
metricTypeMap.put(BYTES_OUT_PER_SEC, classicBytesOutRate);

metricTypeMap.put(BYTES_REJECTED_PER_SEC, new MeterWrapper(BYTES_REJECTED_PER_SEC, "bytes"));
metricTypeMap.put(FAILED_PRODUCE_REQUESTS_PER_SEC, new MeterWrapper(FAILED_PRODUCE_REQUESTS_PER_SEC, "requests"));
metricTypeMap.put(FAILED_FETCH_REQUESTS_PER_SEC, new MeterWrapper(FAILED_FETCH_REQUESTS_PER_SEC, "requests"));
Expand Down Expand Up @@ -131,21 +155,24 @@ private BrokerTopicMetrics(Optional<String> name, boolean remoteStorageEnabled)
public void closeMetric(String metricName) {
MeterWrapper mw = metricTypeMap.get(metricName);
if (mw != null) mw.close();
if (isAllTopicsStats) {
if (BYTES_IN_PER_SEC.equals(metricName) && disklessBytesInRate != null) {
disklessBytesInRate.close();
}
if (BYTES_OUT_PER_SEC.equals(metricName) && disklessBytesOutRate != null) {
disklessBytesOutRate.close();
}
}
GaugeWrapper mg = metricGaugeTypeMap.get(metricName);
if (mg != null) mg.close();

if (BYTES_IN_PER_SEC.equals(metricName)) {
mw = metricTypeMap.get(BYTES_IN_PER_SEC_DISKLESS_TOPIC);
if (mw != null) mw.close();
}
if (BYTES_OUT_PER_SEC.equals(metricName)) {
mw = metricTypeMap.get(BYTES_OUT_PER_SEC_DISKLESS_TOPIC);
if (mw != null) mw.close();
}
}

public void close() {
metricTypeMap.values().forEach(MeterWrapper::close);
if (isAllTopicsStats) {
if (disklessBytesInRate != null) disklessBytesInRate.close();
if (disklessBytesOutRate != null) disklessBytesOutRate.close();
}
metricGaugeTypeMap.values().forEach(GaugeWrapper::close);
}

Expand All @@ -162,22 +189,96 @@ public Meter messagesInRate() {
return metricTypeMap.get(MESSAGE_IN_PER_SEC).meter();
}

/**
* Returns the bytes in rate meter for topic-specific metrics.
* <p>
* This is a convenience method equivalent to calling {@link #bytesInRate(boolean)} with {@code false}.
* It is primarily intended for topic-specific metrics (when a topic name was specified in the constructor),
* where the {@code isDiskless} parameter is ignored and the meter is tagged only by topic name.
* <p>
* For all-topics stats (broker-level aggregation), prefer using {@link #bytesInRate(boolean)} explicitly
* to make the topic type tagging intention clear.
*
* @return the bytes in rate meter
*/
public Meter bytesInRate() {
return bytesInRate(false);
}

/**
* Returns the bytes in rate meter, optionally tagged by topic type.
* <p>
* Behavior depends on whether this is an all-topics aggregated metric or topic-specific:
* <ul>
* <li><b>All-topics stats</b> (when no topic name was specified in constructor):
* Returns a meter tagged with {@code topicType=classic} or {@code topicType=diskless}.
* Separate meters are maintained for each topic type to allow independent tracking
* of classic vs diskless topic throughput at the broker level.</li>
* <li><b>Topic-specific stats</b> (when a topic name was specified in constructor):
* The {@code isDiskless} parameter is ignored. Returns the same meter regardless
* of the parameter value, as topic-specific metrics are only tagged by the topic
* name and do not include a {@code topicType} tag. This is intentional: tagging per-topic
* meters with topicType would double the number of meters we keep in memory (classic + diskless
* variants per topic).</li>
* </ul>
*
* @param isDiskless if true, returns the diskless-tagged meter (for all-topics stats only);
* if false, returns the classic-tagged meter (for all-topics stats only)
* @return the bytes in rate meter
*/
public Meter bytesInRate(boolean isDiskless) {
if (isAllTopicsStats && isDiskless) {
return disklessBytesInRate.meter();
}
// For topic-specific stats, isDiskless is intentionally ignored.
return metricTypeMap.get(BYTES_IN_PER_SEC).meter();
}

public Meter bytesInRateDisklessTopicType() {
return metricTypeMap.get(BYTES_IN_PER_SEC_DISKLESS_TOPIC).meter();
}

/**
* Returns the bytes out rate meter for topic-specific metrics.
* <p>
* This is a convenience method equivalent to calling {@link #bytesOutRate(boolean)} with {@code false}.
* It is primarily intended for topic-specific metrics (when a topic name was specified in the constructor),
* where the {@code isDiskless} parameter is ignored and the meter is tagged only by topic name.
* <p>
* For all-topics stats (broker-level aggregation), prefer using {@link #bytesOutRate(boolean)} explicitly
* to make the topic type tagging intention clear.
*
* @return the bytes out rate meter
*/
public Meter bytesOutRate() {
return bytesOutRate(false);
}

/**
* Returns the bytes out rate meter, optionally tagged by topic type.
* <p>
* Behavior depends on whether this is an all-topics aggregated metric or topic-specific:
* <ul>
* <li><b>All-topics stats</b> (when no topic name was specified in constructor):
* Returns a meter tagged with {@code topicType=classic} or {@code topicType=diskless}.
* Separate meters are maintained for each topic type to allow independent tracking
* of classic vs diskless topic throughput at the broker level.</li>
* <li><b>Topic-specific stats</b> (when a topic name was specified in constructor):
* The {@code isDiskless} parameter is ignored. Returns the same meter regardless
* of the parameter value, as topic-specific metrics are only tagged by the topic
* name and do not include a {@code topicType} tag. This is intentional: tagging per-topic
* meters with topicType would double the number of meters we keep in memory (classic + diskless
* variants per topic).</li>
* </ul>
*
* @param isDiskless if true, returns the diskless-tagged meter (for all-topics stats only);
* if false, returns the classic-tagged meter (for all-topics stats only)
* @return the bytes out rate meter
*/
public Meter bytesOutRate(boolean isDiskless) {
if (isAllTopicsStats && isDiskless) {
return disklessBytesOutRate.meter();
}
// For topic-specific stats, isDiskless is intentionally ignored.
return metricTypeMap.get(BYTES_OUT_PER_SEC).meter();
}

public Meter bytesOutRateDisklessTopicType() {
return metricTypeMap.get(BYTES_OUT_PER_SEC_DISKLESS_TOPIC).meter();
}

public Meter bytesRejectedRate() {
return metricTypeMap.get(BYTES_REJECTED_PER_SEC).meter();
}
Expand Down Expand Up @@ -381,16 +482,12 @@ public MeterWrapper(String metricType, String eventType) {
this(metricType, eventType, BrokerTopicMetrics.this.tags);
}

public MeterWrapper(String metricType,
String eventType,
Map<String, String> metricTags) {
public MeterWrapper(String metricType, String eventType, Map<String, String> metricTags) {
this.metricType = metricType;
this.eventType = eventType;
this.metricTags = new java.util.HashMap<>(metricTags);
this.metricTags = new HashMap<>(metricTags);
if (this.metricTags.isEmpty()) {
meter(); // greedily initialize the general topic metrics
} else if (this.metricTags.size() == 1 && this.metricTags.containsKey("topicType")) {
meter(); // the metrics that only has topicType tag are also global
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,29 +116,18 @@ public void removeMetrics(String topic) {
}

public void updateBytesOut(String topic, boolean isFollower, boolean isReassignment, long value) {
if (isFollower) {
if (isReassignment) {
updateReassignmentBytesOut(value);
}
updateReplicationBytesOut(value);
} else {
topicStats(topic).bytesOutRate().mark(value);
allTopicsStats.bytesOutRate().mark(value);
}
updateBytesOut(topic, isFollower, isReassignment, value, false);
}

/**
* Like {@link #updateBytesOut(String, boolean, boolean, long)}, but for diskless topics.
*/
public void updateBytesOutForDisklessTopic(String topic, boolean isFollower, boolean isReassignment, long value) {
public void updateBytesOut(String topic, boolean isFollower, boolean isReassignment, long value, boolean isDiskless) {
if (isFollower) {
if (isReassignment) {
updateReassignmentBytesOut(value);
}
updateReplicationBytesOut(value);
} else {
topicStats(topic).bytesOutRateDisklessTopicType().mark(value);
allTopicsStats.bytesOutRateDisklessTopicType().mark(value);
topicStats(topic).bytesOutRate().mark(value);
allTopicsStats.bytesOutRate(isDiskless).mark(value);
}
}

Expand Down
Loading
Loading