Browse Source
Implements the following metrics: kafka.server:type=group-coordinator-metrics,name=num-partitions,state=loading kafka.server:type=group-coordinator-metrics,name=num-partitions,state=active kafka.server:type=group-coordinator-metrics,name=num-partitions,state=failed kafka.server:type=group-coordinator-metrics,name=event-queue-size kafka.server:type=group-coordinator-metrics,name=partition-load-time-max kafka.server:type=group-coordinator-metrics,name=partition-load-time-avg kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-min kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-avg The PR makes these metrics generic so that in the future the transaction coordinator runtime can implement the same metrics in a similar fashion. Also, CoordinatorLoaderImpl#load will now return LoadSummary which encapsulates the start time, end time, number of records/bytes. Co-authored-by: David Jacot <djacot@confluent.io> Reviewers: Ritika Reddy <rreddy@confluent.io>, Calvin Liu <caliu@confluent.io>, David Jacot <djacot@confluent.io>, Justine Olshan <jolshan@confluent.io>pull/12449/merge
Jeff Kim
11 months ago
committed by
GitHub
16 changed files with 1124 additions and 51 deletions
@ -0,0 +1,68 @@
@@ -0,0 +1,68 @@
|
||||
/* |
||||
* 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 org.apache.kafka.coordinator.group.metrics; |
||||
|
||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState; |
||||
|
||||
import java.util.function.Supplier; |
||||
|
||||
/** |
||||
* Used by the group and transaction coordinator runtimes, the metrics suite holds partition state gauges and sensors. |
||||
*/ |
||||
public interface CoordinatorRuntimeMetrics extends AutoCloseable { |
||||
|
||||
/** |
||||
* Called when the partition state changes. |
||||
* @param oldState The old state. |
||||
* @param newState The new state to transition to. |
||||
*/ |
||||
void recordPartitionStateChange(CoordinatorState oldState, CoordinatorState newState); |
||||
|
||||
/** |
||||
* Record the partition load metric. |
||||
* @param startTimeMs The partition load start time. |
||||
* @param endTimeMs The partition load end time. |
||||
*/ |
||||
void recordPartitionLoadSensor(long startTimeMs, long endTimeMs); |
||||
|
||||
/** |
||||
* Update the event queue time. |
||||
* |
||||
* @param durationMs The queue time. |
||||
*/ |
||||
void recordEventQueueTime(long durationMs); |
||||
|
||||
/** |
||||
* Update the event queue processing time. |
||||
* |
||||
* @param durationMs The event processing time. |
||||
*/ |
||||
void recordEventQueueProcessingTime(long durationMs); |
||||
|
||||
/** |
||||
* Record the thread idle ratio. |
||||
* @param ratio The idle ratio. |
||||
*/ |
||||
void recordThreadIdleRatio(double ratio); |
||||
|
||||
/** |
||||
* Register the event queue size gauge. |
||||
* |
||||
* @param sizeSupplier The size supplier. |
||||
*/ |
||||
void registerEventQueueSizeGauge(Supplier<Integer> sizeSupplier); |
||||
} |
@ -0,0 +1,219 @@
@@ -0,0 +1,219 @@
|
||||
/* |
||||
* 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 org.apache.kafka.coordinator.group.metrics; |
||||
|
||||
import org.apache.kafka.common.MetricName; |
||||
import org.apache.kafka.common.metrics.Gauge; |
||||
import org.apache.kafka.common.metrics.Metrics; |
||||
import org.apache.kafka.common.metrics.Sensor; |
||||
import org.apache.kafka.common.metrics.stats.Avg; |
||||
import org.apache.kafka.common.metrics.stats.Max; |
||||
import org.apache.kafka.common.metrics.stats.Min; |
||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState; |
||||
|
||||
import java.util.Arrays; |
||||
import java.util.Objects; |
||||
import java.util.concurrent.atomic.AtomicLong; |
||||
import java.util.function.Supplier; |
||||
|
||||
public class GroupCoordinatorRuntimeMetrics implements CoordinatorRuntimeMetrics { |
||||
/** |
||||
* The metrics group. |
||||
*/ |
||||
public static final String METRICS_GROUP = "group-coordinator-metrics"; |
||||
|
||||
/** |
||||
* The partition count metric name. |
||||
*/ |
||||
public static final String NUM_PARTITIONS_METRIC_NAME = "num-partitions"; |
||||
|
||||
/** |
||||
* Metric to count the number of partitions in Loading state. |
||||
*/ |
||||
private final MetricName numPartitionsLoading; |
||||
private final AtomicLong numPartitionsLoadingCounter = new AtomicLong(0); |
||||
|
||||
/** |
||||
* Metric to count the number of partitions in Active state. |
||||
*/ |
||||
private final MetricName numPartitionsActive; |
||||
private final AtomicLong numPartitionsActiveCounter = new AtomicLong(0); |
||||
|
||||
/** |
||||
* Metric to count the number of partitions in Failed state. |
||||
*/ |
||||
private final MetricName numPartitionsFailed; |
||||
private final AtomicLong numPartitionsFailedCounter = new AtomicLong(0); |
||||
|
||||
/** |
||||
* Metric to count the size of the processor queue. |
||||
*/ |
||||
private final MetricName eventQueueSize; |
||||
|
||||
/** |
||||
* The Kafka metrics registry. |
||||
*/ |
||||
private final Metrics metrics; |
||||
|
||||
/** |
||||
* The partition load sensor. |
||||
*/ |
||||
private Sensor partitionLoadSensor; |
||||
|
||||
/** |
||||
* The thread idle sensor. |
||||
*/ |
||||
private Sensor threadIdleRatioSensor; |
||||
|
||||
public GroupCoordinatorRuntimeMetrics(Metrics metrics) { |
||||
this.metrics = Objects.requireNonNull(metrics); |
||||
|
||||
this.numPartitionsLoading = kafkaMetricName( |
||||
NUM_PARTITIONS_METRIC_NAME, |
||||
"The number of partitions in Loading state.", |
||||
"state", "loading" |
||||
); |
||||
|
||||
this.numPartitionsActive = kafkaMetricName( |
||||
NUM_PARTITIONS_METRIC_NAME, |
||||
"The number of partitions in Active state.", |
||||
"state", "active" |
||||
); |
||||
|
||||
this.numPartitionsFailed = kafkaMetricName( |
||||
NUM_PARTITIONS_METRIC_NAME, |
||||
"The number of partitions in Failed state.", |
||||
"state", "failed" |
||||
); |
||||
|
||||
this.eventQueueSize = kafkaMetricName("event-queue-size", "The event accumulator queue size."); |
||||
|
||||
metrics.addMetric(numPartitionsLoading, (Gauge<Long>) (config, now) -> numPartitionsLoadingCounter.get()); |
||||
metrics.addMetric(numPartitionsActive, (Gauge<Long>) (config, now) -> numPartitionsActiveCounter.get()); |
||||
metrics.addMetric(numPartitionsFailed, (Gauge<Long>) (config, now) -> numPartitionsFailedCounter.get()); |
||||
|
||||
this.partitionLoadSensor = metrics.sensor("GroupPartitionLoadTime"); |
||||
this.partitionLoadSensor.add( |
||||
metrics.metricName( |
||||
"partition-load-time-max", |
||||
METRICS_GROUP, |
||||
"The max time it took to load the partitions in the last 30 sec." |
||||
), new Max()); |
||||
this.partitionLoadSensor.add( |
||||
metrics.metricName( |
||||
"partition-load-time-avg", |
||||
METRICS_GROUP, |
||||
"The average time it took to load the partitions in the last 30 sec." |
||||
), new Avg()); |
||||
|
||||
this.threadIdleRatioSensor = metrics.sensor("ThreadIdleRatio"); |
||||
this.threadIdleRatioSensor.add( |
||||
metrics.metricName( |
||||
"thread-idle-ratio-min", |
||||
METRICS_GROUP, |
||||
"The minimum thread idle ratio over the last 30 seconds." |
||||
), new Min()); |
||||
this.threadIdleRatioSensor.add( |
||||
metrics.metricName( |
||||
"thread-idle-ratio-avg", |
||||
METRICS_GROUP, |
||||
"The average thread idle ratio over the last 30 seconds." |
||||
), new Avg()); |
||||
} |
||||
|
||||
/** |
||||
* Retrieve the kafka metric name. |
||||
* |
||||
* @param name The name of the metric. |
||||
* |
||||
* @return The kafka metric name. |
||||
*/ |
||||
private MetricName kafkaMetricName(String name, String description, String... keyValue) { |
||||
return metrics.metricName(name, METRICS_GROUP, description, keyValue); |
||||
} |
||||
|
||||
@Override |
||||
public void close() { |
||||
Arrays.asList( |
||||
numPartitionsLoading, |
||||
numPartitionsActive, |
||||
numPartitionsFailed, |
||||
eventQueueSize |
||||
).forEach(metrics::removeMetric); |
||||
|
||||
metrics.removeSensor(partitionLoadSensor.name()); |
||||
metrics.removeSensor(threadIdleRatioSensor.name()); |
||||
} |
||||
|
||||
/** |
||||
* Called when the partition state changes. Decrement the old state and increment the new state. |
||||
* |
||||
* @param oldState The old state. |
||||
* @param newState The new state to transition to. |
||||
*/ |
||||
@Override |
||||
public void recordPartitionStateChange(CoordinatorState oldState, CoordinatorState newState) { |
||||
switch (oldState) { |
||||
case INITIAL: |
||||
case CLOSED: |
||||
break; |
||||
case LOADING: |
||||
numPartitionsLoadingCounter.decrementAndGet(); |
||||
break; |
||||
case ACTIVE: |
||||
numPartitionsActiveCounter.decrementAndGet(); |
||||
break; |
||||
case FAILED: |
||||
numPartitionsFailedCounter.decrementAndGet(); |
||||
} |
||||
|
||||
switch (newState) { |
||||
case INITIAL: |
||||
case CLOSED: |
||||
break; |
||||
case LOADING: |
||||
numPartitionsLoadingCounter.incrementAndGet(); |
||||
break; |
||||
case ACTIVE: |
||||
numPartitionsActiveCounter.incrementAndGet(); |
||||
break; |
||||
case FAILED: |
||||
numPartitionsFailedCounter.incrementAndGet(); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public void recordPartitionLoadSensor(long startTimeMs, long endTimeMs) { |
||||
this.partitionLoadSensor.record(endTimeMs - startTimeMs, endTimeMs, false); |
||||
} |
||||
|
||||
@Override |
||||
public void recordEventQueueTime(long durationMs) { } |
||||
|
||||
@Override |
||||
public void recordEventQueueProcessingTime(long durationMs) { } |
||||
|
||||
@Override |
||||
public void recordThreadIdleRatio(double ratio) { |
||||
threadIdleRatioSensor.record(ratio); |
||||
} |
||||
|
||||
@Override |
||||
public void registerEventQueueSizeGauge(Supplier<Integer> sizeSupplier) { |
||||
metrics.addMetric(eventQueueSize, (Gauge<Long>) (config, now) -> (long) sizeSupplier.get()); |
||||
} |
||||
} |
@ -0,0 +1,149 @@
@@ -0,0 +1,149 @@
|
||||
/* |
||||
* 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 org.apache.kafka.coordinator.group.metrics; |
||||
|
||||
import org.apache.kafka.common.MetricName; |
||||
import org.apache.kafka.common.metrics.KafkaMetric; |
||||
import org.apache.kafka.common.metrics.Metrics; |
||||
import org.apache.kafka.common.utils.MockTime; |
||||
import org.apache.kafka.common.utils.Time; |
||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState; |
||||
import org.junit.jupiter.api.Test; |
||||
|
||||
import java.util.Arrays; |
||||
import java.util.HashSet; |
||||
import java.util.stream.IntStream; |
||||
|
||||
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.METRICS_GROUP; |
||||
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.NUM_PARTITIONS_METRIC_NAME; |
||||
import static org.junit.jupiter.api.Assertions.assertEquals; |
||||
import static org.junit.jupiter.api.Assertions.assertFalse; |
||||
import static org.junit.jupiter.api.Assertions.assertTrue; |
||||
|
||||
public class GroupCoordinatorRuntimeMetricsTest { |
||||
|
||||
@Test |
||||
public void testMetricNames() { |
||||
Metrics metrics = new Metrics(); |
||||
|
||||
HashSet<org.apache.kafka.common.MetricName> expectedMetrics = new HashSet<>(Arrays.asList( |
||||
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "loading"), |
||||
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "active"), |
||||
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "failed"), |
||||
metrics.metricName("event-queue-size", METRICS_GROUP), |
||||
metrics.metricName("partition-load-time-max", METRICS_GROUP), |
||||
metrics.metricName("partition-load-time-avg", METRICS_GROUP), |
||||
metrics.metricName("thread-idle-ratio-min", METRICS_GROUP), |
||||
metrics.metricName("thread-idle-ratio-avg", METRICS_GROUP) |
||||
)); |
||||
|
||||
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) { |
||||
runtimeMetrics.registerEventQueueSizeGauge(() -> 0); |
||||
expectedMetrics.forEach(metricName -> assertTrue(metrics.metrics().containsKey(metricName))); |
||||
} |
||||
|
||||
expectedMetrics.forEach(metricName -> assertFalse(metrics.metrics().containsKey(metricName))); |
||||
} |
||||
|
||||
@Test |
||||
public void testUpdateNumPartitionsMetrics() { |
||||
Metrics metrics = new Metrics(); |
||||
|
||||
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) { |
||||
IntStream.range(0, 10) |
||||
.forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.INITIAL, CoordinatorState.LOADING)); |
||||
IntStream.range(0, 8) |
||||
.forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.LOADING, CoordinatorState.ACTIVE)); |
||||
IntStream.range(0, 8) |
||||
.forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.ACTIVE, CoordinatorState.FAILED)); |
||||
IntStream.range(0, 2) |
||||
.forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.FAILED, CoordinatorState.CLOSED)); |
||||
|
||||
assertMetricGauge(metrics, kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "loading"), 2); |
||||
assertMetricGauge(metrics, kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "active"), 0); |
||||
assertMetricGauge(metrics, kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "failed"), 6); |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testPartitionLoadSensorMetrics() { |
||||
Time time = new MockTime(); |
||||
Metrics metrics = new Metrics(time); |
||||
|
||||
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) { |
||||
long startTimeMs = time.milliseconds(); |
||||
runtimeMetrics.recordPartitionLoadSensor(startTimeMs, startTimeMs + 1000); |
||||
runtimeMetrics.recordPartitionLoadSensor(startTimeMs, startTimeMs + 2000); |
||||
|
||||
org.apache.kafka.common.MetricName metricName = metrics.metricName( |
||||
"partition-load-time-avg", METRICS_GROUP); |
||||
|
||||
KafkaMetric metric = metrics.metrics().get(metricName); |
||||
assertEquals(1500.0, metric.metricValue()); |
||||
|
||||
metricName = metrics.metricName( |
||||
"partition-load-time-max", METRICS_GROUP); |
||||
metric = metrics.metrics().get(metricName); |
||||
assertEquals(2000.0, metric.metricValue()); |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testThreadIdleRatioSensor() { |
||||
Time time = new MockTime(); |
||||
Metrics metrics = new Metrics(time); |
||||
|
||||
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) { |
||||
IntStream.range(0, 3).forEach(i -> runtimeMetrics.recordThreadIdleRatio(1.0 / (i + 1))); |
||||
|
||||
org.apache.kafka.common.MetricName metricName = metrics.metricName( |
||||
"thread-idle-ratio-avg", METRICS_GROUP); |
||||
|
||||
KafkaMetric metric = metrics.metrics().get(metricName); |
||||
assertEquals((11.0 / 6.0) / 3.0, metric.metricValue()); // (6/6 + 3/6 + 2/6) / 3
|
||||
|
||||
metricName = metrics.metricName( |
||||
"thread-idle-ratio-min", METRICS_GROUP); |
||||
metric = metrics.metrics().get(metricName); |
||||
assertEquals(1.0 / 3.0, metric.metricValue()); |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testEventQueueSize() { |
||||
Time time = new MockTime(); |
||||
Metrics metrics = new Metrics(time); |
||||
|
||||
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) { |
||||
runtimeMetrics.registerEventQueueSizeGauge(() -> 5); |
||||
assertMetricGauge(metrics, kafkaMetricName(metrics, "event-queue-size"), 5); |
||||
} |
||||
} |
||||
|
||||
private static void assertMetricGauge(Metrics metrics, org.apache.kafka.common.MetricName metricName, long count) { |
||||
assertEquals(count, (long) metrics.metric(metricName).metricValue()); |
||||
} |
||||
|
||||
private static com.yammer.metrics.core.MetricName yammerMetricName(String type, String name) { |
||||
String mBeanName = String.format("kafka.coordinator.group:type=%s,name=%s", type, name); |
||||
return new com.yammer.metrics.core.MetricName("kafka.coordinator.group", type, name, null, mBeanName); |
||||
} |
||||
|
||||
private static MetricName kafkaMetricName(Metrics metrics, String name, String... keyValue) { |
||||
return metrics.metricName(name, METRICS_GROUP, "", keyValue); |
||||
} |
||||
} |
Loading…
Reference in new issue