Skip to content
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

KAFKA-15189: only init remote topic metrics when enabled #14133

Merged
merged 14 commits into from Aug 5, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -171,7 +171,7 @@ public KafkaApis build() {
if (metrics == null) throw new RuntimeException("You must set metrics");
if (quotas == null) throw new RuntimeException("You must set quotas");
if (fetchManager == null) throw new RuntimeException("You must set fetchManager");
if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats();
if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().enableRemoteStorageSystem());
if (apiVersionManager == null) throw new RuntimeException("You must set apiVersionManager");

return new KafkaApis(requestChannel,
Expand Down
Expand Up @@ -54,7 +54,7 @@ public class ReplicaManagerBuilder {
private MetadataCache metadataCache = null;
private LogDirFailureChannel logDirFailureChannel = null;
private AlterPartitionManager alterPartitionManager = null;
private BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
private BrokerTopicStats brokerTopicStats = null;
private AtomicBoolean isShuttingDown = new AtomicBoolean(false);
private Optional<RemoteLogManager> remoteLogManager = Optional.empty();
private Optional<KafkaZkClient> zkClient = Optional.empty();
Expand Down Expand Up @@ -179,6 +179,7 @@ public ReplicaManager build() {
if (metadataCache == null) throw new RuntimeException("You must set metadataCache");
if (logDirFailureChannel == null) throw new RuntimeException("You must set logDirFailureChannel");
if (alterPartitionManager == null) throw new RuntimeException("You must set alterIsrManager");
if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().enableRemoteStorageSystem());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Initializing brokerTopicStats if null, with the initialized config in L176 above.

return new ReplicaManager(config,
metrics,
time,
Expand Down
3 changes: 1 addition & 2 deletions core/src/main/scala/kafka/server/BrokerServer.scala
Expand Up @@ -188,8 +188,7 @@ class BrokerServer(
kafkaScheduler.startup()

/* register broker metrics */
brokerTopicStats = new BrokerTopicStats

brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.enableRemoteStorageSystem())

quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-")

Expand Down
40 changes: 22 additions & 18 deletions core/src/main/scala/kafka/server/KafkaRequestHandler.scala
Expand Up @@ -227,7 +227,7 @@ class KafkaRequestHandlerPool(val brokerId: Int,
}
}

class BrokerTopicMetrics(name: Option[String]) {
class BrokerTopicMetrics(name: Option[String], systemRemoteStorageEnabled: Boolean) {
showuon marked this conversation as resolved.
Show resolved Hide resolved
private val metricsGroup = new KafkaMetricsGroup(this.getClass)

val tags: java.util.Map[String, String] = name match {
Expand Down Expand Up @@ -277,17 +277,21 @@ class BrokerTopicMetrics(name: Option[String]) {
BrokerTopicStats.TotalFetchRequestsPerSec -> MeterWrapper(BrokerTopicStats.TotalFetchRequestsPerSec, "requests"),
BrokerTopicStats.FetchMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.FetchMessageConversionsPerSec, "requests"),
BrokerTopicStats.ProduceMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.ProduceMessageConversionsPerSec, "requests"),
BrokerTopicStats.RemoteBytesOutPerSec -> MeterWrapper(BrokerTopicStats.RemoteBytesOutPerSec, "bytes"),
BrokerTopicStats.RemoteBytesInPerSec -> MeterWrapper(BrokerTopicStats.RemoteBytesInPerSec, "bytes"),
BrokerTopicStats.RemoteReadRequestsPerSec -> MeterWrapper(BrokerTopicStats.RemoteReadRequestsPerSec, "requests"),
BrokerTopicStats.RemoteWriteRequestsPerSec -> MeterWrapper(BrokerTopicStats.RemoteWriteRequestsPerSec, "requests"),
BrokerTopicStats.FailedRemoteReadRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedRemoteReadRequestsPerSec, "requests"),
BrokerTopicStats.FailedRemoteWriteRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedRemoteWriteRequestsPerSec, "requests"),
BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec -> MeterWrapper(BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec, "requests"),
BrokerTopicStats.InvalidMagicNumberRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMagicNumberRecordsPerSec, "requests"),
BrokerTopicStats.InvalidMessageCrcRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMessageCrcRecordsPerSec, "requests"),
BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec, "requests")
).asJava)
if (systemRemoteStorageEnabled) {
metricTypeMap.putAll(Map(
BrokerTopicStats.RemoteBytesOutPerSec -> MeterWrapper(BrokerTopicStats.RemoteBytesOutPerSec, "bytes"),
BrokerTopicStats.RemoteBytesInPerSec -> MeterWrapper(BrokerTopicStats.RemoteBytesInPerSec, "bytes"),
BrokerTopicStats.RemoteReadRequestsPerSec -> MeterWrapper(BrokerTopicStats.RemoteReadRequestsPerSec, "requests"),
BrokerTopicStats.RemoteWriteRequestsPerSec -> MeterWrapper(BrokerTopicStats.RemoteWriteRequestsPerSec, "requests"),
BrokerTopicStats.FailedRemoteReadRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedRemoteReadRequestsPerSec, "requests"),
BrokerTopicStats.FailedRemoteWriteRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedRemoteWriteRequestsPerSec, "requests")
).asJava)
}
if (name.isEmpty) {
metricTypeMap.put(BrokerTopicStats.ReplicationBytesInPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesInPerSec, "bytes"))
metricTypeMap.put(BrokerTopicStats.ReplicationBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesOutPerSec, "bytes"))
Expand Down Expand Up @@ -390,15 +394,13 @@ object BrokerTopicStats {
val InvalidMagicNumberRecordsPerSec = "InvalidMagicNumberRecordsPerSec"
val InvalidMessageCrcRecordsPerSec = "InvalidMessageCrcRecordsPerSec"
val InvalidOffsetOrSequenceRecordsPerSec = "InvalidOffsetOrSequenceRecordsPerSec"

private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k))
}

class BrokerTopicStats extends Logging {
import BrokerTopicStats._
class BrokerTopicStats(systemRemoteStorageEnabled: Boolean = false) extends Logging {

private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k), systemRemoteStorageEnabled)
private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory))
val allTopicsStats = new BrokerTopicMetrics(None)
val allTopicsStats = new BrokerTopicMetrics(None, systemRemoteStorageEnabled)

def topicStats(topic: String): BrokerTopicMetrics =
stats.getAndMaybePut(topic)
Expand Down Expand Up @@ -439,12 +441,14 @@ class BrokerTopicStats extends Logging {
topicMetrics.closeMetric(BrokerTopicStats.ProduceMessageConversionsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ReplicationBytesOutPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ReassignmentBytesOutPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteBytesOutPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteBytesInPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteReadRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteWriteRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.FailedRemoteReadRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.FailedRemoteWriteRequestsPerSec)
if (systemRemoteStorageEnabled) {
showuon marked this conversation as resolved.
Show resolved Hide resolved
topicMetrics.closeMetric(BrokerTopicStats.RemoteBytesOutPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteBytesInPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteReadRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.RemoteWriteRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.FailedRemoteReadRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.FailedRemoteWriteRequestsPerSec)
}
}
}

Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/KafkaServer.scala
Expand Up @@ -261,7 +261,7 @@ class KafkaServer(
metrics = Server.initializeMetrics(config, time, clusterId)

/* register broker metrics */
_brokerTopicStats = new BrokerTopicStats
_brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.enableRemoteStorageSystem())

quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse(""))
KafkaBroker.notifyClusterListeners(clusterId, kafkaMetricsReporters ++ metrics.reporters.asScala)
Expand Down
Expand Up @@ -127,7 +127,7 @@ public class RemoteLogManagerTest {
RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class);
RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class);
RemoteLogManagerConfig remoteLogManagerConfig = null;
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
BrokerTopicStats brokerTopicStats = new BrokerTopicStats(true);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

force to true since we're testing RemoteLogManager

RemoteLogManager remoteLogManager = null;

TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("Leader", 0));
Expand Down
Expand Up @@ -44,7 +44,7 @@
public class RemoteLogReaderTest {
public static final String TOPIC = "test";
RemoteLogManager mockRLM = mock(RemoteLogManager.class);
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
BrokerTopicStats brokerTopicStats = new BrokerTopicStats(true);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

force to true since we're testing RemoteLogReader

LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100);
Records records = mock(Records.class);

Expand Down
28 changes: 27 additions & 1 deletion core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala
Expand Up @@ -25,8 +25,10 @@ import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.{RequestContext, RequestHeader}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.MockTime
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.mockito.ArgumentMatchers
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{mock, when}
Expand Down Expand Up @@ -77,4 +79,28 @@ class KafkaRequestHandlerTest {
assertEquals(Some(startTime + 2000000), request.callbackRequestDequeueTimeNanos)
assertEquals(Some(startTime + 3000000), request.callbackRequestCompleteTimeNanos)
}


@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testTopicStats(systemRemoteStorageEnabled: Boolean): Unit = {
divijvaidya marked this conversation as resolved.
Show resolved Hide resolved
val topic = "topic"
val brokerTopicStats = new BrokerTopicStats(systemRemoteStorageEnabled)
brokerTopicStats.topicStats(topic)
if (systemRemoteStorageEnabled) {
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteBytesInPerSec))
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteBytesOutPerSec))
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteReadRequestsPerSec))
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteWriteRequestsPerSec))
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.FailedRemoteReadRequestsPerSec))
assertTrue(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.FailedRemoteWriteRequestsPerSec))
} else {
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteBytesInPerSec))
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteBytesOutPerSec))
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteReadRequestsPerSec))
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.RemoteWriteRequestsPerSec))
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.FailedRemoteReadRequestsPerSec))
assertFalse(brokerTopicStats.topicStats(topic).metricMap.contains(BrokerTopicStats.FailedRemoteWriteRequestsPerSec))
}
}
}