chia7712 commented on code in PR #19285: URL: https://github.com/apache/kafka/pull/19285#discussion_r2019137257
########## storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteListOffsets.java: ########## @@ -0,0 +1,198 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.requests.ListOffsetsResponse; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.storage.internals.log.OffsetResultHolder; + +import com.yammer.metrics.core.Meter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +public class DelayedRemoteListOffsets extends DelayedOperation { + + private static final Logger LOG = LoggerFactory.getLogger(DelayedRemoteListOffsets.class); + + private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedRemoteListOffsetsMetrics"); Review Comment: Could you please add comments to avoid we change it unintentionally in the future. ########## storage/src/main/java/org/apache/kafka/server/purgatory/PartitionChecker.java: ########## @@ -0,0 +1,33 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ApiException; + +/** + * Interface used to decouple DelayedRemoteListOffsets and ReplicaManager. + */ +public interface PartitionChecker { Review Comment: I'm thinking whether this interface used to decouple should be a individual class - The side effect is that we may introduce many such classes in migrating those delay operations. For example: #19226. or we can use `Consumer<TopicPartition>` instead? ########## storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteListOffsets.java: ########## @@ -0,0 +1,198 @@ +/* + * 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.server.purgatory; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.requests.ListOffsetsResponse; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.storage.internals.log.OffsetResultHolder; + +import com.yammer.metrics.core.Meter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +public class DelayedRemoteListOffsets extends DelayedOperation { + + private static final Logger LOG = LoggerFactory.getLogger(DelayedRemoteListOffsets.class); + + private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedRemoteListOffsetsMetrics"); + final Meter aggregateExpirationMeter = metricsGroup.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); + final Map<TopicPartition, Meter> partitionExpirationMeters = new ConcurrentHashMap<>(); Review Comment: This PR changes the `partitionExpirationMeters` from global object to class member, so is it necessary to use `ConcurrentHashMap`? ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -1506,7 +1506,7 @@ class ReplicaManager(val config: KafkaConfig, if (delayedRemoteListOffsetsRequired(statusByPartition)) { val delayMs: Long = if (timeoutMs > 0) timeoutMs else config.remoteLogManagerConfig.remoteListOffsetsRequestTimeoutMs() // create delayed remote list offsets operation - val delayedRemoteListOffsets = new DelayedRemoteListOffsets(delayMs, version, statusByPartition, this, responseCallback) + val delayedRemoteListOffsets = new DelayedRemoteListOffsets(delayMs, version, statusByPartition.asJava, this, responseCallback) Review Comment: maybe we can replace `this` by `getPartitionOrException`? -- 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