dajac commented on code in PR #14417: URL: https://github.com/apache/kafka/pull/14417#discussion_r1341657675
########## core/src/test/scala/unit/kafka/coordinator/group/CoordinatorLoaderImplTest.scala: ########## @@ -283,7 +292,63 @@ class CoordinatorLoaderImplTest { minOneMessage = true )).thenReturn(readResult) - assertNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)) + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)) + } + } + + @Test + def testLoadSummary(): Unit = { + val tp = new TopicPartition("foo", 0) + val replicaManager = mock(classOf[ReplicaManager]) + val serde = new StringKeyValueDeserializer + val log = mock(classOf[UnifiedLog]) + val coordinator = mock(classOf[CoordinatorPlayback[(String, String)]]) + val time = new MockTime() + + TestUtils.resource(new CoordinatorLoaderImpl[(String, String)]( + time, + replicaManager = replicaManager, + deserializer = serde, + loadBufferSize = 1000 + )) { loader => + val startTimeMs = time.milliseconds() + when(replicaManager.getLog(tp)).thenReturn(Some(log)) + when(log.logStartOffset).thenReturn(0L) + when(replicaManager.getLogEndOffset(tp)).thenReturn(Some(5L)) + + val readResult1 = logReadResult(startOffset = 0, records = Seq( + new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes) + )) + + when(log.read( + startOffset = 0L, + maxLength = 1000, + isolation = FetchIsolation.LOG_END, + minOneMessage = true + )).thenAnswer((_: InvocationOnMock) => { + time.sleep(1000) Review Comment: Why do we need to sleep here? ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); Review Comment: We only use this on from the tests. Should we remove it from the interface? ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(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); + + /** + * Get the counter for partitions in Loading state. + * Only used for testing. + */ + long numPartitionsLoading(); + + /** + * Get the counter for partitions in Active state. + * Only used for testing. + */ + long numPartitionsActive(); + + /** + * Get the counter for partitions in Failed state. + * Only used for testing. + */ + long numPartitionsFailed(); Review Comment: Those should not be in the interface in my opinion. ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(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); + + /** + * Get the counter for partitions in Loading state. + * Only used for testing. + */ + long numPartitionsLoading(); + + /** + * Get the counter for partitions in Active state. + * Only used for testing. + */ + long numPartitionsActive(); + + /** + * Get the counter for partitions in Failed state. + * Only used for testing. + */ + long numPartitionsFailed(); + + /** + * Update the event queue time. + * + * @param durationMs The queue time. + */ + void updateEventQueueTime(long durationMs); + + /** + * Update the event queue processing time. + * + * @param durationMs The event processing time. + */ + void updateEventQueueProcessingTime(long durationMs); + + /** + * Record the failed event. + */ + void recordFailedEvent(); + + /** + * Record the successful event. + */ + void recordSuccessfulEvent(); Review Comment: I wonder if those bring any value. How do you see them? ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,353 @@ +/* + * 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 com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Histogram; +import com.yammer.metrics.core.MetricName; +import com.yammer.metrics.core.MetricsRegistry; +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.Meter; +import org.apache.kafka.common.metrics.stats.Min; +import org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; + +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; + +public class GroupCoordinatorRuntimeMetrics implements CoordinatorRuntimeMetrics { + /** + * The metrics group. + */ + public static final String METRICS_GROUP = "group-coordinator-metrics"; + + /** + * Metric to count the number of partitions in Loading state. + */ + private static final MetricName NUM_PARTITIONS_LOADING = getMetricName( + "CoordinatorRuntime", "NumPartitionsLoading"); Review Comment: I wonder if we should just have all the metrics in `GroupCoordinator`. The issue with `CoordinatorRuntime` is that we could have issues if we have another instance of the runtime as some point. ########## checkstyle/import-control.xml: ########## @@ -239,6 +239,16 @@ <allow pkg="org.apache.kafka.server.util"/> <allow pkg="org.apache.kafka.test" /> <allow pkg="org.apache.kafka.timeline" /> + <subpackage name="metrics"> + <allow pkg="com.yammer.metrics"/> + <allow pkg="org.apache.kafka.common.metrics" /> + <allow pkg="org.apache.kafka.server.metrics" /> + </subpackage> + <subpackage name="runtime"> + <allow pkg="com.yammer.metrics"/> + <allow pkg="org.apache.kafka.common.metrics" /> + <allow pkg="org.apache.kafka.server.metrics" /> + </subpackage> Review Comment: Why do we need this? ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(CoordinatorState oldState, CoordinatorState newState); Review Comment: nit: `recordPartitionStateChange` to be consistent with the other methods? ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(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); + + /** + * Get the counter for partitions in Loading state. + * Only used for testing. + */ + long numPartitionsLoading(); + + /** + * Get the counter for partitions in Active state. + * Only used for testing. + */ + long numPartitionsActive(); + + /** + * Get the counter for partitions in Failed state. + * Only used for testing. + */ + long numPartitionsFailed(); + + /** + * Update the event queue time. + * + * @param durationMs The queue time. + */ + void updateEventQueueTime(long durationMs); Review Comment: nit: record* ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(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); + + /** + * Get the counter for partitions in Loading state. + * Only used for testing. + */ + long numPartitionsLoading(); + + /** + * Get the counter for partitions in Active state. + * Only used for testing. + */ + long numPartitionsActive(); + + /** + * Get the counter for partitions in Failed state. + * Only used for testing. + */ + long numPartitionsFailed(); + + /** + * Update the event queue time. + * + * @param durationMs The queue time. + */ + void updateEventQueueTime(long durationMs); + + /** + * Update the event queue processing time. + * + * @param durationMs The event processing time. + */ + void updateEventQueueProcessingTime(long durationMs); Review Comment: ditto ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,353 @@ +/* + * 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 com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Histogram; +import com.yammer.metrics.core.MetricName; +import com.yammer.metrics.core.MetricsRegistry; +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.Meter; +import org.apache.kafka.common.metrics.stats.Min; +import org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; + +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; + +public class GroupCoordinatorRuntimeMetrics implements CoordinatorRuntimeMetrics { + /** + * The metrics group. + */ + public static final String METRICS_GROUP = "group-coordinator-metrics"; + + /** + * Metric to count the number of partitions in Loading state. + */ + private static final MetricName NUM_PARTITIONS_LOADING = getMetricName( + "CoordinatorRuntime", "NumPartitionsLoading"); + + /** + * The Loading state partition counter. + */ + private final AtomicLong numPartitionsLoading = new AtomicLong(0); + + /** + * Metric to count the number of partitions in Active state. + */ + private static final MetricName NUM_PARTITIONS_ACTIVE = getMetricName( + "CoordinatorRuntime", "NumPartitionsActive"); + + /** + * The Active state partition counter. + */ + private final AtomicLong numPartitionsActive = new AtomicLong(0); + + /** + * Metric to count the number of partitions in Failed state. + */ + private static final MetricName NUM_PARTITIONS_FAILED = getMetricName( + "CoordinatorRuntime", "NumPartitionsFailed"); + + /** + * The Failed state partition counter. + */ + private final AtomicLong numPartitionsFailed = new AtomicLong(0); + + /** + * Metric to count the size of the processor queue. + */ + private static final MetricName EVENT_QUEUE_SIZE = getMetricName( + "MultiThreadedEventProcessor", "EventQueueSize"); Review Comment: ditto about the name. MultiThreadedEventProcessor is an internal detail. ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/CoordinatorRuntimeMetrics.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 { + /** + * Returns the metrics group. + */ + String metricsGroup(); + + /** + * Called when the partition state changes. + * @param oldState The old state. + * @param newState The new state to transition to. + */ + void onPartitionStateChange(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); + + /** + * Get the counter for partitions in Loading state. + * Only used for testing. + */ + long numPartitionsLoading(); + + /** + * Get the counter for partitions in Active state. + * Only used for testing. + */ + long numPartitionsActive(); + + /** + * Get the counter for partitions in Failed state. + * Only used for testing. + */ + long numPartitionsFailed(); + + /** + * Update the event queue time. + * + * @param durationMs The queue time. + */ + void updateEventQueueTime(long durationMs); + + /** + * Update the event queue processing time. + * + * @param durationMs The event processing time. + */ + void updateEventQueueProcessingTime(long durationMs); + + /** + * Record the failed event. + */ + void recordFailedEvent(); + + /** + * Record the successful event. + */ + void recordSuccessfulEvent(); + + /** + * Record the thread idle ratio. + * @param ratio The idle ratio. + */ + void recordThreadIdleRatio(double ratio); Review Comment: Does it work when you have multiple threads as well? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org