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( | ||||||
|  | @ -40,14 +44,18 @@ public final class QuorumControllerMetrics implements ControllerMetrics { | ||||||
|         "KafkaController", "OfflinePartitionsCount"); |         "KafkaController", "OfflinePartitionsCount"); | ||||||
|     private final static MetricName PREFERRED_REPLICA_IMBALANCE_COUNT = getMetricName( |     private final static MetricName PREFERRED_REPLICA_IMBALANCE_COUNT = getMetricName( | ||||||
|         "KafkaController", "PreferredReplicaImbalanceCount"); |         "KafkaController", "PreferredReplicaImbalanceCount"); | ||||||
| 
 |      | ||||||
|     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