mimaison commented on code in PR #19226:
URL: https://github.com/apache/kafka/pull/19226#discussion_r2012290418


##########
server-common/src/main/java/org/apache/kafka/server/common/DelayedDeleteRecords.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.common;

Review Comment:
   Would the `org.apache.kafka.server.purgatory` package be a better place for 
these classes? So it's next to the abstract class `DelayedOperation` it's 
extending. 



##########
server-common/src/main/java/org/apache/kafka/server/common/DelayedDeleteRecords.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.common;
+
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.metrics.KafkaMetricsGroup;
+import org.apache.kafka.server.purgatory.DelayedOperation;
+
+import com.yammer.metrics.core.Meter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * A delayed delete records operation that can be created by the replica 
manager and watched
+ * in the delete records operation purgatory
+ */
+public class DelayedDeleteRecords extends DelayedOperation {
+    
+    private static final Logger LOG = 
LoggerFactory.getLogger(DelayedDeleteRecords.class);

Review Comment:
   Should we keep `kafka.server.DelayedDeleteRecords` as the logger name?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -19,13 +19,14 @@ package kafka.server
 import com.yammer.metrics.core.Meter
 import kafka.cluster.{Partition, PartitionListener}
 import kafka.controller.StateChangeLogger
-import kafka.log.remote.RemoteLogManager
 import kafka.log.LogManager
+import kafka.log.remote.RemoteLogManager
 import kafka.server.HostedPartition.Online
 import kafka.server.QuotaFactory.QuotaManagers
-import kafka.server.ReplicaManager.{AtMinIsrPartitionCountMetricName, 
FailedIsrUpdatesPerSecMetricName, IsrExpandsPerSecMetricName, 
IsrShrinksPerSecMetricName, LeaderCountMetricName, 
OfflineReplicaCountMetricName, PartitionCountMetricName, 
PartitionsWithLateTransactionsCountMetricName, ProducerIdCountMetricName, 
ReassigningPartitionsMetricName, UnderMinIsrPartitionCountMetricName, 
UnderReplicatedPartitionsMetricName, createLogReadResult, 
isListOffsetsTimestampUnsupported}
+import kafka.server.ReplicaManager._

Review Comment:
   Let's keep the explicit imports



-- 
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

Reply via email to