mirror of https://github.com/apache/kafka.git
KAFKA-12697: Add FencedBrokerCount and ActiveBrokerCount metrics to the QuorumController (#10772)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This commit is contained in:
parent
86e83de742
commit
3f433c0b4a
|
|
@ -116,6 +116,11 @@ public class ClusterControlManager {
|
||||||
*/
|
*/
|
||||||
private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
|
private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A reference to the controller's metrics registry.
|
||||||
|
*/
|
||||||
|
private final ControllerMetrics controllerMetrics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The broker heartbeat manager, or null if this controller is on standby.
|
* The broker heartbeat manager, or null if this controller is on standby.
|
||||||
*/
|
*/
|
||||||
|
|
@ -131,7 +136,8 @@ public class ClusterControlManager {
|
||||||
Time time,
|
Time time,
|
||||||
SnapshotRegistry snapshotRegistry,
|
SnapshotRegistry snapshotRegistry,
|
||||||
long sessionTimeoutNs,
|
long sessionTimeoutNs,
|
||||||
ReplicaPlacer replicaPlacer) {
|
ReplicaPlacer replicaPlacer,
|
||||||
|
ControllerMetrics metrics) {
|
||||||
this.logContext = logContext;
|
this.logContext = logContext;
|
||||||
this.log = logContext.logger(ClusterControlManager.class);
|
this.log = logContext.logger(ClusterControlManager.class);
|
||||||
this.time = time;
|
this.time = time;
|
||||||
|
|
@ -140,6 +146,7 @@ public class ClusterControlManager {
|
||||||
this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
|
this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
this.heartbeatManager = null;
|
this.heartbeatManager = null;
|
||||||
this.readyBrokersFuture = Optional.empty();
|
this.readyBrokersFuture = Optional.empty();
|
||||||
|
this.controllerMetrics = metrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -249,11 +256,13 @@ public class ClusterControlManager {
|
||||||
features.put(feature.name(), new VersionRange(
|
features.put(feature.name(), new VersionRange(
|
||||||
feature.minSupportedVersion(), feature.maxSupportedVersion()));
|
feature.minSupportedVersion(), feature.maxSupportedVersion()));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Update broker registrations.
|
// Update broker registrations.
|
||||||
BrokerRegistration prevRegistration = brokerRegistrations.put(brokerId,
|
BrokerRegistration prevRegistration = brokerRegistrations.put(brokerId,
|
||||||
new BrokerRegistration(brokerId, record.brokerEpoch(),
|
new BrokerRegistration(brokerId, record.brokerEpoch(),
|
||||||
record.incarnationId(), listeners, features,
|
record.incarnationId(), listeners, features,
|
||||||
Optional.ofNullable(record.rack()), record.fenced()));
|
Optional.ofNullable(record.rack()), record.fenced()));
|
||||||
|
updateMetrics(prevRegistration, brokerRegistrations.get(brokerId));
|
||||||
if (prevRegistration == null) {
|
if (prevRegistration == null) {
|
||||||
log.info("Registered new broker: {}", record);
|
log.info("Registered new broker: {}", record);
|
||||||
} else if (prevRegistration.incarnationId().equals(record.incarnationId())) {
|
} else if (prevRegistration.incarnationId().equals(record.incarnationId())) {
|
||||||
|
|
@ -274,6 +283,7 @@ public class ClusterControlManager {
|
||||||
"registration with that epoch found", record.toString()));
|
"registration with that epoch found", record.toString()));
|
||||||
} else {
|
} else {
|
||||||
brokerRegistrations.remove(brokerId);
|
brokerRegistrations.remove(brokerId);
|
||||||
|
updateMetrics(registration, brokerRegistrations.get(brokerId));
|
||||||
log.info("Unregistered broker: {}", record);
|
log.info("Unregistered broker: {}", record);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -289,6 +299,7 @@ public class ClusterControlManager {
|
||||||
"registration with that epoch found", record.toString()));
|
"registration with that epoch found", record.toString()));
|
||||||
} else {
|
} else {
|
||||||
brokerRegistrations.put(brokerId, registration.cloneWithFencing(true));
|
brokerRegistrations.put(brokerId, registration.cloneWithFencing(true));
|
||||||
|
updateMetrics(registration, brokerRegistrations.get(brokerId));
|
||||||
log.info("Fenced broker: {}", record);
|
log.info("Fenced broker: {}", record);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -304,6 +315,7 @@ public class ClusterControlManager {
|
||||||
"registration with that epoch found", record.toString()));
|
"registration with that epoch found", record.toString()));
|
||||||
} else {
|
} else {
|
||||||
brokerRegistrations.put(brokerId, registration.cloneWithFencing(false));
|
brokerRegistrations.put(brokerId, registration.cloneWithFencing(false));
|
||||||
|
updateMetrics(registration, brokerRegistrations.get(brokerId));
|
||||||
log.info("Unfenced broker: {}", record);
|
log.info("Unfenced broker: {}", record);
|
||||||
}
|
}
|
||||||
if (readyBrokersFuture.isPresent()) {
|
if (readyBrokersFuture.isPresent()) {
|
||||||
|
|
@ -314,6 +326,31 @@ public class ClusterControlManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void updateMetrics(BrokerRegistration prevRegistration, BrokerRegistration registration) {
|
||||||
|
if (registration == null) {
|
||||||
|
if (prevRegistration.fenced()) {
|
||||||
|
controllerMetrics.setFencedBrokerCount(controllerMetrics.fencedBrokerCount() - 1);
|
||||||
|
} else {
|
||||||
|
controllerMetrics.setActiveBrokerCount(controllerMetrics.activeBrokerCount() - 1);
|
||||||
|
}
|
||||||
|
} else if (prevRegistration == null) {
|
||||||
|
if (registration.fenced()) {
|
||||||
|
controllerMetrics.setFencedBrokerCount(controllerMetrics.fencedBrokerCount() + 1);
|
||||||
|
} else {
|
||||||
|
controllerMetrics.setActiveBrokerCount(controllerMetrics.activeBrokerCount() + 1);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (prevRegistration.fenced() && !registration.fenced()) {
|
||||||
|
controllerMetrics.setFencedBrokerCount(controllerMetrics.fencedBrokerCount() - 1);
|
||||||
|
controllerMetrics.setActiveBrokerCount(controllerMetrics.activeBrokerCount() + 1);
|
||||||
|
} else if (!prevRegistration.fenced() && registration.fenced()) {
|
||||||
|
controllerMetrics.setFencedBrokerCount(controllerMetrics.fencedBrokerCount() + 1);
|
||||||
|
controllerMetrics.setActiveBrokerCount(controllerMetrics.activeBrokerCount() - 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public List<List<Integer>> placeReplicas(int startPartition,
|
public List<List<Integer>> placeReplicas(int startPartition,
|
||||||
int numPartitions,
|
int numPartitions,
|
||||||
short numReplicas) {
|
short numReplicas) {
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,14 @@ public interface ControllerMetrics extends AutoCloseable {
|
||||||
|
|
||||||
void updateEventQueueProcessingTime(long durationMs);
|
void updateEventQueueProcessingTime(long durationMs);
|
||||||
|
|
||||||
|
void setFencedBrokerCount(int brokerCount);
|
||||||
|
|
||||||
|
int fencedBrokerCount();
|
||||||
|
|
||||||
|
void setActiveBrokerCount(int brokerCount);
|
||||||
|
|
||||||
|
int activeBrokerCount();
|
||||||
|
|
||||||
void setGlobalTopicsCount(int topicCount);
|
void setGlobalTopicsCount(int topicCount);
|
||||||
|
|
||||||
int globalTopicsCount();
|
int globalTopicsCount();
|
||||||
|
|
|
||||||
|
|
@ -1125,7 +1125,7 @@ public final class QuorumController implements Controller {
|
||||||
snapshotRegistry, configDefs, alterConfigPolicy);
|
snapshotRegistry, configDefs, alterConfigPolicy);
|
||||||
this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry);
|
this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry);
|
||||||
this.clusterControl = new ClusterControlManager(logContext, time,
|
this.clusterControl = new ClusterControlManager(logContext, time,
|
||||||
snapshotRegistry, sessionTimeoutNs, replicaPlacer);
|
snapshotRegistry, sessionTimeoutNs, replicaPlacer, controllerMetrics);
|
||||||
this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry);
|
this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry);
|
||||||
this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry);
|
this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry);
|
||||||
this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes;
|
this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes;
|
||||||
|
|
|
||||||
|
|
@ -32,6 +32,10 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
|
||||||
"ControllerEventManager", "EventQueueTimeMs");
|
"ControllerEventManager", "EventQueueTimeMs");
|
||||||
private final static MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName(
|
private final static MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName(
|
||||||
"ControllerEventManager", "EventQueueProcessingTimeMs");
|
"ControllerEventManager", "EventQueueProcessingTimeMs");
|
||||||
|
private final static MetricName FENCED_BROKER_COUNT = getMetricName(
|
||||||
|
"KafkaController", "FencedBrokerCount");
|
||||||
|
private final static MetricName ACTIVE_BROKER_COUNT = getMetricName(
|
||||||
|
"KafkaController", "ActiveBrokerCount");
|
||||||
private final static MetricName GLOBAL_TOPIC_COUNT = getMetricName(
|
private final static MetricName GLOBAL_TOPIC_COUNT = getMetricName(
|
||||||
"KafkaController", "GlobalTopicCount");
|
"KafkaController", "GlobalTopicCount");
|
||||||
private final static MetricName GLOBAL_PARTITION_COUNT = getMetricName(
|
private final static MetricName GLOBAL_PARTITION_COUNT = getMetricName(
|
||||||
|
|
@ -43,11 +47,15 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
|
||||||
|
|
||||||
private final MetricsRegistry registry;
|
private final MetricsRegistry registry;
|
||||||
private volatile boolean active;
|
private volatile boolean active;
|
||||||
|
private volatile int fencedBrokerCount;
|
||||||
|
private volatile int activeBrokerCount;
|
||||||
private volatile int globalTopicCount;
|
private volatile int globalTopicCount;
|
||||||
private volatile int globalPartitionCount;
|
private volatile int globalPartitionCount;
|
||||||
private volatile int offlinePartitionCount;
|
private volatile int offlinePartitionCount;
|
||||||
private volatile int preferredReplicaImbalanceCount;
|
private volatile int preferredReplicaImbalanceCount;
|
||||||
private final Gauge<Integer> activeControllerCount;
|
private final Gauge<Integer> activeControllerCount;
|
||||||
|
private final Gauge<Integer> fencedBrokerCountGauge;
|
||||||
|
private final Gauge<Integer> activeBrokerCountGauge;
|
||||||
private final Gauge<Integer> globalPartitionCountGauge;
|
private final Gauge<Integer> globalPartitionCountGauge;
|
||||||
private final Gauge<Integer> globalTopicCountGauge;
|
private final Gauge<Integer> globalTopicCountGauge;
|
||||||
private final Gauge<Integer> offlinePartitionCountGauge;
|
private final Gauge<Integer> offlinePartitionCountGauge;
|
||||||
|
|
@ -58,6 +66,8 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
|
||||||
public QuorumControllerMetrics(MetricsRegistry registry) {
|
public QuorumControllerMetrics(MetricsRegistry registry) {
|
||||||
this.registry = Objects.requireNonNull(registry);
|
this.registry = Objects.requireNonNull(registry);
|
||||||
this.active = false;
|
this.active = false;
|
||||||
|
this.fencedBrokerCount = 0;
|
||||||
|
this.activeBrokerCount = 0;
|
||||||
this.globalTopicCount = 0;
|
this.globalTopicCount = 0;
|
||||||
this.globalPartitionCount = 0;
|
this.globalPartitionCount = 0;
|
||||||
this.offlinePartitionCount = 0;
|
this.offlinePartitionCount = 0;
|
||||||
|
|
@ -70,6 +80,18 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
|
||||||
});
|
});
|
||||||
this.eventQueueTime = registry.newHistogram(EVENT_QUEUE_TIME_MS, true);
|
this.eventQueueTime = registry.newHistogram(EVENT_QUEUE_TIME_MS, true);
|
||||||
this.eventQueueProcessingTime = registry.newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true);
|
this.eventQueueProcessingTime = registry.newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true);
|
||||||
|
this.fencedBrokerCountGauge = registry.newGauge(FENCED_BROKER_COUNT, new Gauge<Integer>() {
|
||||||
|
@Override
|
||||||
|
public Integer value() {
|
||||||
|
return fencedBrokerCount;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
this.activeBrokerCountGauge = registry.newGauge(ACTIVE_BROKER_COUNT, new Gauge<Integer>() {
|
||||||
|
@Override
|
||||||
|
public Integer value() {
|
||||||
|
return activeBrokerCount;
|
||||||
|
}
|
||||||
|
});
|
||||||
this.globalTopicCountGauge = registry.newGauge(GLOBAL_TOPIC_COUNT, new Gauge<Integer>() {
|
this.globalTopicCountGauge = registry.newGauge(GLOBAL_TOPIC_COUNT, new Gauge<Integer>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer value() {
|
public Integer value() {
|
||||||
|
|
@ -116,6 +138,25 @@ public final class QuorumControllerMetrics implements ControllerMetrics {
|
||||||
eventQueueTime.update(durationMs);
|
eventQueueTime.update(durationMs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setFencedBrokerCount(int brokerCount) {
|
||||||
|
this.fencedBrokerCount = brokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int fencedBrokerCount() {
|
||||||
|
return this.fencedBrokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setActiveBrokerCount(int brokerCount) {
|
||||||
|
this.activeBrokerCount = brokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int activeBrokerCount() {
|
||||||
|
return this.activeBrokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setGlobalTopicsCount(int topicCount) {
|
public void setGlobalTopicsCount(int topicCount) {
|
||||||
this.globalTopicCount = topicCount;
|
this.globalTopicCount = topicCount;
|
||||||
|
|
|
||||||
|
|
@ -58,7 +58,7 @@ public class ClusterControlManagerTest {
|
||||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||||
ClusterControlManager clusterControl = new ClusterControlManager(
|
ClusterControlManager clusterControl = new ClusterControlManager(
|
||||||
new LogContext(), time, snapshotRegistry, 1000,
|
new LogContext(), time, snapshotRegistry, 1000,
|
||||||
new StripedReplicaPlacer(new Random()));
|
new StripedReplicaPlacer(new Random()), new MockControllerMetrics());
|
||||||
clusterControl.activate();
|
clusterControl.activate();
|
||||||
assertFalse(clusterControl.unfenced(0));
|
assertFalse(clusterControl.unfenced(0));
|
||||||
|
|
||||||
|
|
@ -99,7 +99,7 @@ public class ClusterControlManagerTest {
|
||||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||||
ClusterControlManager clusterControl = new ClusterControlManager(
|
ClusterControlManager clusterControl = new ClusterControlManager(
|
||||||
new LogContext(), new MockTime(0, 0, 0), snapshotRegistry, 1000,
|
new LogContext(), new MockTime(0, 0, 0), snapshotRegistry, 1000,
|
||||||
new StripedReplicaPlacer(new Random()));
|
new StripedReplicaPlacer(new Random()), new MockControllerMetrics());
|
||||||
clusterControl.activate();
|
clusterControl.activate();
|
||||||
clusterControl.replay(brokerRecord);
|
clusterControl.replay(brokerRecord);
|
||||||
assertEquals(new BrokerRegistration(1, 100,
|
assertEquals(new BrokerRegistration(1, 100,
|
||||||
|
|
@ -122,7 +122,7 @@ public class ClusterControlManagerTest {
|
||||||
MockRandom random = new MockRandom();
|
MockRandom random = new MockRandom();
|
||||||
ClusterControlManager clusterControl = new ClusterControlManager(
|
ClusterControlManager clusterControl = new ClusterControlManager(
|
||||||
new LogContext(), time, snapshotRegistry, 1000,
|
new LogContext(), time, snapshotRegistry, 1000,
|
||||||
new StripedReplicaPlacer(random));
|
new StripedReplicaPlacer(random), new MockControllerMetrics());
|
||||||
clusterControl.activate();
|
clusterControl.activate();
|
||||||
for (int i = 0; i < numUsableBrokers; i++) {
|
for (int i = 0; i < numUsableBrokers; i++) {
|
||||||
RegisterBrokerRecord brokerRecord =
|
RegisterBrokerRecord brokerRecord =
|
||||||
|
|
@ -159,7 +159,7 @@ public class ClusterControlManagerTest {
|
||||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||||
ClusterControlManager clusterControl = new ClusterControlManager(
|
ClusterControlManager clusterControl = new ClusterControlManager(
|
||||||
new LogContext(), time, snapshotRegistry, 1000,
|
new LogContext(), time, snapshotRegistry, 1000,
|
||||||
new StripedReplicaPlacer(new Random()));
|
new StripedReplicaPlacer(new Random()), new MockControllerMetrics());
|
||||||
clusterControl.activate();
|
clusterControl.activate();
|
||||||
assertFalse(clusterControl.unfenced(0));
|
assertFalse(clusterControl.unfenced(0));
|
||||||
for (int i = 0; i < 3; i++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,8 @@ package org.apache.kafka.controller;
|
||||||
|
|
||||||
public final class MockControllerMetrics implements ControllerMetrics {
|
public final class MockControllerMetrics implements ControllerMetrics {
|
||||||
private volatile boolean active;
|
private volatile boolean active;
|
||||||
|
private volatile int fencedBrokers;
|
||||||
|
private volatile int activeBrokers;
|
||||||
private volatile int topics;
|
private volatile int topics;
|
||||||
private volatile int partitions;
|
private volatile int partitions;
|
||||||
private volatile int offlinePartitions;
|
private volatile int offlinePartitions;
|
||||||
|
|
@ -27,6 +29,8 @@ public final class MockControllerMetrics implements ControllerMetrics {
|
||||||
|
|
||||||
public MockControllerMetrics() {
|
public MockControllerMetrics() {
|
||||||
this.active = false;
|
this.active = false;
|
||||||
|
this.fencedBrokers = 0;
|
||||||
|
this.activeBrokers = 0;
|
||||||
this.topics = 0;
|
this.topics = 0;
|
||||||
this.partitions = 0;
|
this.partitions = 0;
|
||||||
this.offlinePartitions = 0;
|
this.offlinePartitions = 0;
|
||||||
|
|
@ -53,6 +57,26 @@ public final class MockControllerMetrics implements ControllerMetrics {
|
||||||
// nothing to do
|
// nothing to do
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setFencedBrokerCount(int brokerCount) {
|
||||||
|
this.fencedBrokers = brokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int fencedBrokerCount() {
|
||||||
|
return this.fencedBrokers;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setActiveBrokerCount(int brokerCount) {
|
||||||
|
this.activeBrokers = brokerCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int activeBrokerCount() {
|
||||||
|
return activeBrokers;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setGlobalTopicsCount(int topicCount) {
|
public void setGlobalTopicsCount(int topicCount) {
|
||||||
this.topics = topicCount;
|
this.topics = topicCount;
|
||||||
|
|
|
||||||
|
|
@ -54,7 +54,7 @@ public class ProducerIdControlManagerTest {
|
||||||
snapshotRegistry = new SnapshotRegistry(logContext);
|
snapshotRegistry = new SnapshotRegistry(logContext);
|
||||||
clusterControl = new ClusterControlManager(
|
clusterControl = new ClusterControlManager(
|
||||||
logContext, time, snapshotRegistry, 1000,
|
logContext, time, snapshotRegistry, 1000,
|
||||||
new StripedReplicaPlacer(random));
|
new StripedReplicaPlacer(random), new MockControllerMetrics());
|
||||||
|
|
||||||
clusterControl.activate();
|
clusterControl.activate();
|
||||||
for (int i = 0; i < 4; i++) {
|
for (int i = 0; i < 4; i++) {
|
||||||
|
|
|
||||||
|
|
@ -132,10 +132,10 @@ public class ReplicationControlManagerTest {
|
||||||
final LogContext logContext = new LogContext();
|
final LogContext logContext = new LogContext();
|
||||||
final MockTime time = new MockTime();
|
final MockTime time = new MockTime();
|
||||||
final MockRandom random = new MockRandom();
|
final MockRandom random = new MockRandom();
|
||||||
|
final ControllerMetrics metrics = new MockControllerMetrics();
|
||||||
final ClusterControlManager clusterControl = new ClusterControlManager(
|
final ClusterControlManager clusterControl = new ClusterControlManager(
|
||||||
logContext, time, snapshotRegistry, TimeUnit.MILLISECONDS.convert(BROKER_SESSION_TIMEOUT_MS, TimeUnit.NANOSECONDS),
|
logContext, time, snapshotRegistry, TimeUnit.MILLISECONDS.convert(BROKER_SESSION_TIMEOUT_MS, TimeUnit.NANOSECONDS),
|
||||||
new StripedReplicaPlacer(random));
|
new StripedReplicaPlacer(random), metrics);
|
||||||
final ControllerMetrics metrics = new MockControllerMetrics();
|
|
||||||
final ConfigurationControlManager configurationControl = new ConfigurationControlManager(
|
final ConfigurationControlManager configurationControl = new ConfigurationControlManager(
|
||||||
new LogContext(), snapshotRegistry, Collections.emptyMap(), Optional.empty());
|
new LogContext(), snapshotRegistry, Collections.emptyMap(), Optional.empty());
|
||||||
final ReplicationControlManager replicationControl;
|
final ReplicationControlManager replicationControl;
|
||||||
|
|
@ -429,6 +429,41 @@ public class ReplicationControlManagerTest {
|
||||||
ctx.replicationControl.iterator(Long.MAX_VALUE));
|
ctx.replicationControl.iterator(Long.MAX_VALUE));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBrokerCountMetrics() throws Exception {
|
||||||
|
ReplicationControlTestContext ctx = new ReplicationControlTestContext();
|
||||||
|
ReplicationControlManager replicationControl = ctx.replicationControl;
|
||||||
|
|
||||||
|
ctx.registerBrokers(0);
|
||||||
|
|
||||||
|
assertEquals(1, ctx.metrics.fencedBrokerCount());
|
||||||
|
assertEquals(0, ctx.metrics.activeBrokerCount());
|
||||||
|
|
||||||
|
ctx.unfenceBrokers(0);
|
||||||
|
|
||||||
|
assertEquals(0, ctx.metrics.fencedBrokerCount());
|
||||||
|
assertEquals(1, ctx.metrics.activeBrokerCount());
|
||||||
|
|
||||||
|
ctx.registerBrokers(1);
|
||||||
|
ctx.unfenceBrokers(1);
|
||||||
|
|
||||||
|
assertEquals(2, ctx.metrics.activeBrokerCount());
|
||||||
|
|
||||||
|
ctx.registerBrokers(2);
|
||||||
|
ctx.unfenceBrokers(2);
|
||||||
|
|
||||||
|
assertEquals(0, ctx.metrics.fencedBrokerCount());
|
||||||
|
assertEquals(3, ctx.metrics.activeBrokerCount());
|
||||||
|
|
||||||
|
ControllerResult<Void> result = replicationControl.unregisterBroker(0);
|
||||||
|
ctx.replay(result.records());
|
||||||
|
result = replicationControl.unregisterBroker(2);
|
||||||
|
ctx.replay(result.records());
|
||||||
|
|
||||||
|
assertEquals(0, ctx.metrics.fencedBrokerCount());
|
||||||
|
assertEquals(1, ctx.metrics.activeBrokerCount());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateTopicsWithValidateOnlyFlag() throws Exception {
|
public void testCreateTopicsWithValidateOnlyFlag() throws Exception {
|
||||||
ReplicationControlTestContext ctx = new ReplicationControlTestContext();
|
ReplicationControlTestContext ctx = new ReplicationControlTestContext();
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue