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 4 commits
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
66 changes: 66 additions & 0 deletions core/src/main/java/kafka/log/remote/RemoteStorageMetrics.java
@@ -0,0 +1,66 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log.remote;

import com.yammer.metrics.core.MetricName;
import kafka.server.BrokerTopicStats;
import org.apache.kafka.server.metrics.KafkaYammerMetrics;

import java.util.HashSet;
import java.util.Set;

import static org.apache.kafka.storage.internals.log.RemoteStorageThreadPool.AVG_IDLE_PERCENT;
import static org.apache.kafka.storage.internals.log.RemoteStorageThreadPool.TASK_QUEUE_SIZE;

public class RemoteStorageMetrics {
showuon marked this conversation as resolved.
Show resolved Hide resolved
final static MetricName REMOTE_BYTES_OUT_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.RemoteBytesOutPerSec());
final static MetricName REMOTE_BYTES_IN_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.RemoteBytesInPerSec());
final static MetricName REMOTE_READ_REQUESTS_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.RemoteReadRequestsPerSec());
final static MetricName REMOTE_WRITE_REQUESTS_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.RemoteWriteRequestsPerSec());
final static MetricName FAILED_REMOTE_READ_REQUESTS_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.FailedRemoteReadRequestsPerSec());
final static MetricName FAILED_REMOTE_WRITE_REQUESTS_PER_SEC = getMetricName(
"kafka.server", "BrokerTopicMetrics", BrokerTopicStats.FailedRemoteWriteRequestsPerSec());
final static MetricName REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT = getMetricName(
"kafka.log.remote", "RemoteLogManager", RemoteLogManager.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT);
final static MetricName REMOTE_LOG_READER_TASK_QUEUE_SIZE = getMetricName(
"org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", RemoteLogManager.REMOTE_LOG_READER_METRICS_NAME_PREFIX + TASK_QUEUE_SIZE);
divijvaidya marked this conversation as resolved.
Show resolved Hide resolved
final static MetricName REMOTE_LOG_READER_AVG_IDLE_PERCENT = getMetricName(
"org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", RemoteLogManager.REMOTE_LOG_READER_METRICS_NAME_PREFIX + AVG_IDLE_PERCENT);

public static Set<MetricName> allMetrics() {
Set<MetricName> metrics = new HashSet<>();
metrics.add(REMOTE_BYTES_OUT_PER_SEC);
metrics.add(REMOTE_BYTES_IN_PER_SEC);
metrics.add(REMOTE_READ_REQUESTS_PER_SEC);
metrics.add(REMOTE_WRITE_REQUESTS_PER_SEC);
metrics.add(FAILED_REMOTE_READ_REQUESTS_PER_SEC);
metrics.add(FAILED_REMOTE_WRITE_REQUESTS_PER_SEC);
metrics.add(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT);
metrics.add(REMOTE_LOG_READER_AVG_IDLE_PERCENT);
metrics.add(REMOTE_LOG_READER_TASK_QUEUE_SIZE);

return metrics;
}
private static MetricName getMetricName(String group, String type, String name) {
return KafkaYammerMetrics.getMetricName(group, type, name);
}
}
Expand Up @@ -38,6 +38,7 @@

import java.util.Collections;
import java.util.Optional;

showuon marked this conversation as resolved.
Show resolved Hide resolved
import scala.compat.java8.OptionConverters;


Expand Down Expand Up @@ -171,7 +172,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(Optional.of(config));
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(Optional.of(config));
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(java.util.Optional.of(config))

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

Expand Down
29 changes: 17 additions & 12 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], configOpt: java.util.Optional[KafkaConfig]) {
private val metricsGroup = new KafkaMetricsGroup(this.getClass)

val tags: java.util.Map[String, String] = name match {
Expand Down Expand Up @@ -277,17 +277,24 @@ 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)

configOpt.ifPresent(config =>
if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) {
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 +397,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(configOpt: java.util.Optional[KafkaConfig] = java.util.Optional.empty()) extends Logging {

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

def topicStats(topic: String): BrokerTopicMetrics =
stats.getAndMaybePut(topic)
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(java.util.Optional.of(config))

quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse(""))
KafkaBroker.notifyClusterListeners(clusterId, kafkaMetricsReporters ++ metrics.reporters.asScala)
Expand Down
7 changes: 5 additions & 2 deletions core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java
Expand Up @@ -127,7 +127,8 @@ public class RemoteLogManagerTest {
RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class);
RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class);
RemoteLogManagerConfig remoteLogManagerConfig = null;
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();

BrokerTopicStats brokerTopicStats = null;
RemoteLogManager remoteLogManager = null;

TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("Leader", 0));
Expand Down Expand Up @@ -157,8 +158,10 @@ public List<EpochEntry> read() {
void setUp() throws Exception {
topicIds.put(leaderTopicIdPartition.topicPartition().topic(), leaderTopicIdPartition.topicId());
topicIds.put(followerTopicIdPartition.topicPartition().topic(), followerTopicIdPartition.topicId());
Properties props = new Properties();
Properties props = kafka.utils.TestUtils.createDummyBrokerConfig();
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true");
remoteLogManagerConfig = createRLMConfig(props);
brokerTopicStats = new BrokerTopicStats(Optional.of(KafkaConfig.fromProps(props)));

kafka.utils.TestUtils.clearYammerMetrics();

Expand Down
9 changes: 8 additions & 1 deletion core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java
Expand Up @@ -17,9 +17,11 @@
package kafka.log.remote;

import kafka.server.BrokerTopicStats;
import kafka.server.KafkaConfig;
import kafka.utils.TestUtils;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig;
import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
import org.apache.kafka.storage.internals.log.FetchDataInfo;
import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
Expand All @@ -30,6 +32,8 @@
import org.mockito.ArgumentCaptor;

import java.io.IOException;
import java.util.Optional;
import java.util.Properties;
import java.util.function.Consumer;

import static org.junit.jupiter.api.Assertions.assertEquals;
Expand All @@ -44,13 +48,16 @@
public class RemoteLogReaderTest {
public static final String TOPIC = "test";
RemoteLogManager mockRLM = mock(RemoteLogManager.class);
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
BrokerTopicStats brokerTopicStats = null;
LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100);
Records records = mock(Records.class);

@BeforeEach
public void setUp() {
TestUtils.clearYammerMetrics();
Properties props = kafka.utils.TestUtils.createDummyBrokerConfig();
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true");
brokerTopicStats = new BrokerTopicStats(Optional.of(KafkaConfig.fromProps(props)));
}

@Test
Expand Down
31 changes: 28 additions & 3 deletions core/src/test/scala/integration/kafka/api/MetricsTest.scala
Expand Up @@ -16,6 +16,7 @@ import java.util.{Locale, Properties}
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{JaasTestUtils, TestUtils}
import com.yammer.metrics.core.{Gauge, Histogram, Meter}
import kafka.log.remote.RemoteStorageMetrics
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.{Metric, MetricName, TopicPartition}
Expand All @@ -24,9 +25,12 @@ import org.apache.kafka.common.errors.{InvalidTopicException, UnknownTopicOrPart
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.authenticator.TestJaasConfig
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource

import scala.annotation.nowarn
import scala.jdk.CollectionConverters._
Expand Down Expand Up @@ -54,6 +58,12 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {

@BeforeEach
override def setUp(testInfo: TestInfo): Unit = {
if (testInfo.getDisplayName.endsWith("true")) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Wouldn't this enable RemoteStorage for all tests which use @Parameterized (and not just for testMetrics)? An alternative way is to add the full test name here OR check for string "systemRemoteStorageEnabled: 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.

You're right! Currently there's only 1 test in this test suite, but we should make it much clear. Updating now.

// systemRemoteStorageEnabled is enabled
this.serverConfig.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true")
this.serverConfig.setProperty(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteStorageManager].getName)
this.serverConfig.setProperty(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName)
}
verifyNoRequestMetrics("Request metrics not removed in a previous test")
startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, kafkaServerJaasEntryName))
super.setUp(testInfo)
Expand All @@ -70,8 +80,9 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
* Verifies some of the metrics of producer, consumer as well as server.
*/
@nowarn("cat=deprecation")
@Test
def testMetrics(): Unit = {
@ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {0}")
@ValueSource(booleans = Array(true, false))
def testMetrics(systemRemoteStorageEnabled: Boolean): Unit = {
val topic = "topicWithOldMessageFormat"
val props = new Properties
props.setProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0")
Expand Down Expand Up @@ -103,6 +114,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {

generateAuthenticationFailure(tp)
verifyBrokerAuthenticationMetrics(server)
verifyRemoteStorageMetrics(systemRemoteStorageEnabled)
}

private def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int,
Expand Down Expand Up @@ -308,4 +320,17 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
}
assertTrue(metrics.isEmpty, s"$errorMessage: ${metrics.keys}")
}

private def verifyRemoteStorageMetrics(shouldContainMetrics: Boolean): Unit = {
val metrics = RemoteStorageMetrics.allMetrics().asScala.filter(name =>
KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.find(metric => {
metric._1.getMBeanName().equals(name.getMBeanName)
}).isDefined
).toList
if (shouldContainMetrics) {
assertEquals(RemoteStorageMetrics.allMetrics().size(), metrics.size, s"Only $metrics appear in the metrics")
} else {
assertEquals(0, metrics.size, s"$metrics should not appear in the metrics")
}
}
}
31 changes: 30 additions & 1 deletion core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala
Expand Up @@ -25,8 +25,11 @@ 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.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
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 +80,30 @@ 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 props = kafka.utils.TestUtils.createDummyBrokerConfig()
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, systemRemoteStorageEnabled.toString)
val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props)))
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))
}
}
}