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


##########
server-common/src/main/java/org/apache/kafka/server/common/DeleteRecordsPartitionStatus.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class DeleteRecordsPartitionStatus {
+    private final long requiredOffset;
+    private final DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus;
+    private volatile boolean acksPending;
+
+    public DeleteRecordsPartitionStatus(long requiredOffset,
+                                        
DeleteRecordsResponseData.DeleteRecordsPartitionResult responseStatus) {
+        this.requiredOffset = requiredOffset;
+        this.responseStatus = responseStatus;
+        this.acksPending = false;
+    }
+
+    public boolean acksPending() {
+        return acksPending;
+    }
+
+    public void setAcksPending(boolean acksPending) {
+        this.acksPending = acksPending;
+    }
+
+
+    public DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus() {
+        return responseStatus;
+    }
+
+    public long requiredOffset() {
+        return requiredOffset;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("[acksPending: %b, error: %s, lowWatermark: %d, 
requiredOffset: %d]",
+                acksPending, 
Errors.forCode(responseStatus.errorCode()).toString(), 
responseStatus.lowWatermark(),
+                requiredOffset);
+    }
+
+    /**
+     * A delayed delete records operation that can be created by the replica 
manager and watched
+     * in the delete records operation purgatory
+     */
+    public static class DelayedDeleteRecords extends DelayedOperation {
+        private final Map<TopicPartition, DeleteRecordsPartitionStatus> 
deleteRecordsStatus;
+        private final BiConsumer<TopicPartition, DeleteRecordsPartitionStatus> 
onAcksPending;
+        private final Consumer<Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback;
+        private static final Logger log = 
LoggerFactory.getLogger(DelayedDeleteRecords.class);
+
+        public DelayedDeleteRecords(long delayMs,
+                                    Map<TopicPartition, 
DeleteRecordsPartitionStatus> deleteRecordsStatus,
+                                    //  To maintain compatibility with 
dependency packages, the logic has been moved to the caller.
+                                    BiConsumer<TopicPartition, 
DeleteRecordsPartitionStatus> onAcksPending,
+                                    Consumer<Map<TopicPartition,
+                                            
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback) {
+            super(delayMs);
+            this.onAcksPending = onAcksPending;
+            this.deleteRecordsStatus = new 
ConcurrentHashMap<>(deleteRecordsStatus);
+            this.responseCallback = responseCallback;
+            // first update the acks pending variable according to the error 
code
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.responseStatus().errorCode() == Errors.NONE.code()) 
{
+                    // Timeout error state will be cleared when required acks 
are received
+                    status.setAcksPending(true);
+                    
status.responseStatus().setErrorCode(Errors.REQUEST_TIMED_OUT.code());
+                } else {
+                    status.setAcksPending(false);
+                }
+            });
+        }
+
+        /**
+         * The delayed delete records operation can be completed if every 
partition specified in the request satisfied one of the following:
+         *
+         * 1) There was an error while checking if all replicas have caught up 
to the deleteRecordsOffset: set an error in response
+         * 2) The low watermark of the partition has caught up to the 
deleteRecordsOffset. set the low watermark in response
+         *
+         */
+        @Override
+        public boolean tryComplete() {
+            //  check for each partition if it still has pending acks
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                log.trace("Checking delete records satisfaction for {}, 
current status {}", topicPartition, status);
+                //  skip those partitions that have already been satisfied
+                if (status.acksPending()) {
+                    onAcksPending.accept(topicPartition, status);
+                }
+            });
+            //  check if every partition has satisfied at least one of case A 
or B
+            return 
deleteRecordsStatus.values().stream().noneMatch(DeleteRecordsPartitionStatus::acksPending)
 && forceComplete();
+        }
+
+        @Override
+        public void onExpiration() {
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.acksPending()) {
+                    
DelayedDeleteRecordsMetrics.recordExpiration(topicPartition);
+                }
+            });
+        }
+
+        /**
+         * Upon completion, return the current response status along with the 
error code per partition
+         */
+        @Override
+        public void onComplete() {
+            Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult> responseStatus =
+                    new HashMap<>();
+            deleteRecordsStatus.forEach((k, status) -> {
+                responseStatus.put(k, status.responseStatus());
+            });
+            responseCallback.accept(responseStatus);
+        }
+    }
+
+    private static class DelayedDeleteRecordsMetrics {

Review Comment:
   we don't need this static class as all static variables can be moved to 
`DelayedDeleteRecords`



##########
server-common/src/main/java/org/apache/kafka/server/common/DeleteRecordsPartitionStatus.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class DeleteRecordsPartitionStatus {
+    private final long requiredOffset;
+    private final DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus;
+    private volatile boolean acksPending;
+
+    public DeleteRecordsPartitionStatus(long requiredOffset,
+                                        
DeleteRecordsResponseData.DeleteRecordsPartitionResult responseStatus) {
+        this.requiredOffset = requiredOffset;
+        this.responseStatus = responseStatus;
+        this.acksPending = false;
+    }
+
+    public boolean acksPending() {
+        return acksPending;
+    }
+
+    public void setAcksPending(boolean acksPending) {
+        this.acksPending = acksPending;
+    }
+
+
+    public DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus() {
+        return responseStatus;
+    }
+
+    public long requiredOffset() {
+        return requiredOffset;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("[acksPending: %b, error: %s, lowWatermark: %d, 
requiredOffset: %d]",
+                acksPending, 
Errors.forCode(responseStatus.errorCode()).toString(), 
responseStatus.lowWatermark(),
+                requiredOffset);
+    }
+
+    /**
+     * A delayed delete records operation that can be created by the replica 
manager and watched
+     * in the delete records operation purgatory
+     */
+    public static class DelayedDeleteRecords extends DelayedOperation {
+        private final Map<TopicPartition, DeleteRecordsPartitionStatus> 
deleteRecordsStatus;
+        private final BiConsumer<TopicPartition, DeleteRecordsPartitionStatus> 
onAcksPending;
+        private final Consumer<Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback;
+        private static final Logger log = 
LoggerFactory.getLogger(DelayedDeleteRecords.class);
+
+        public DelayedDeleteRecords(long delayMs,
+                                    Map<TopicPartition, 
DeleteRecordsPartitionStatus> deleteRecordsStatus,
+                                    //  To maintain compatibility with 
dependency packages, the logic has been moved to the caller.
+                                    BiConsumer<TopicPartition, 
DeleteRecordsPartitionStatus> onAcksPending,
+                                    Consumer<Map<TopicPartition,
+                                            
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback) {
+            super(delayMs);
+            this.onAcksPending = onAcksPending;
+            this.deleteRecordsStatus = new 
ConcurrentHashMap<>(deleteRecordsStatus);
+            this.responseCallback = responseCallback;
+            // first update the acks pending variable according to the error 
code
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.responseStatus().errorCode() == Errors.NONE.code()) 
{
+                    // Timeout error state will be cleared when required acks 
are received
+                    status.setAcksPending(true);
+                    
status.responseStatus().setErrorCode(Errors.REQUEST_TIMED_OUT.code());
+                } else {
+                    status.setAcksPending(false);
+                }
+            });
+        }
+
+        /**
+         * The delayed delete records operation can be completed if every 
partition specified in the request satisfied one of the following:
+         *
+         * 1) There was an error while checking if all replicas have caught up 
to the deleteRecordsOffset: set an error in response
+         * 2) The low watermark of the partition has caught up to the 
deleteRecordsOffset. set the low watermark in response
+         *
+         */
+        @Override
+        public boolean tryComplete() {
+            //  check for each partition if it still has pending acks
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                log.trace("Checking delete records satisfaction for {}, 
current status {}", topicPartition, status);
+                //  skip those partitions that have already been satisfied
+                if (status.acksPending()) {
+                    onAcksPending.accept(topicPartition, status);
+                }
+            });
+            //  check if every partition has satisfied at least one of case A 
or B
+            return 
deleteRecordsStatus.values().stream().noneMatch(DeleteRecordsPartitionStatus::acksPending)
 && forceComplete();
+        }
+
+        @Override
+        public void onExpiration() {
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.acksPending()) {
+                    
DelayedDeleteRecordsMetrics.recordExpiration(topicPartition);
+                }
+            });
+        }
+
+        /**
+         * Upon completion, return the current response status along with the 
error code per partition
+         */
+        @Override
+        public void onComplete() {
+            Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult> responseStatus =
+                    new HashMap<>();
+            deleteRecordsStatus.forEach((k, status) -> {
+                responseStatus.put(k, status.responseStatus());
+            });
+            responseCallback.accept(responseStatus);

Review Comment:
   ```java
   
responseCallback.accept(deleteRecordsStatus.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey,
 e -> e.getValue().responseStatus())));
   ```



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1282,8 +1282,33 @@ class ReplicaManager(val config: KafkaConfig,
     }
 
     if (delayedDeleteRecordsRequired(localDeleteRecordsResults)) {
+      def onAcks(topicPartition: TopicPartition, status: 
DeleteRecordsPartitionStatus): Unit = {
+        val (lowWatermarkReached, error, lw) = getPartition(topicPartition) 
match {
+          case HostedPartition.Online(partition) =>
+            partition.leaderLogIfLocal match {
+              case Some(_) =>
+                val leaderLW = partition.lowWatermarkIfLeader
+                (leaderLW >= status.requiredOffset, Errors.NONE, leaderLW)
+              case None =>
+                (false, Errors.NOT_LEADER_OR_FOLLOWER, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+            }
+
+          case HostedPartition.Offline(_) =>
+            (false, Errors.KAFKA_STORAGE_ERROR, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+
+          case HostedPartition.None =>
+            (false, Errors.UNKNOWN_TOPIC_OR_PARTITION, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+        }
+        if (error != Errors.NONE || lowWatermarkReached) {
+          status.setAcksPending(false)
+          status.responseStatus.setErrorCode(error.code)
+          status.responseStatus.setLowWatermark(lw)
+        }
+      }
+      val responseCallbackJava: 
java.util.function.Consumer[util.Map[TopicPartition, 
DeleteRecordsPartitionResult]] =
+        response => responseCallback(response.asScala)
       // create delayed delete records operation
-      val delayedDeleteRecords = new DelayedDeleteRecords(timeout, 
deleteRecordsStatus, this, responseCallback)
+      val delayedDeleteRecords = new 
DeleteRecordsPartitionStatus.DelayedDeleteRecords(timeout, 
deleteRecordsStatus.asJava,onAcks , responseCallbackJava)

Review Comment:
   `deleteRecordsStatus.asJava, onAcks, responseCallbackJava`



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1282,8 +1282,33 @@ class ReplicaManager(val config: KafkaConfig,
     }
 
     if (delayedDeleteRecordsRequired(localDeleteRecordsResults)) {
+      def onAcks(topicPartition: TopicPartition, status: 
DeleteRecordsPartitionStatus): Unit = {
+        val (lowWatermarkReached, error, lw) = getPartition(topicPartition) 
match {
+          case HostedPartition.Online(partition) =>
+            partition.leaderLogIfLocal match {
+              case Some(_) =>
+                val leaderLW = partition.lowWatermarkIfLeader
+                (leaderLW >= status.requiredOffset, Errors.NONE, leaderLW)
+              case None =>
+                (false, Errors.NOT_LEADER_OR_FOLLOWER, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+            }
+
+          case HostedPartition.Offline(_) =>
+            (false, Errors.KAFKA_STORAGE_ERROR, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+
+          case HostedPartition.None =>
+            (false, Errors.UNKNOWN_TOPIC_OR_PARTITION, 
DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+        }
+        if (error != Errors.NONE || lowWatermarkReached) {
+          status.setAcksPending(false)
+          status.responseStatus.setErrorCode(error.code)
+          status.responseStatus.setLowWatermark(lw)
+        }
+      }
+      val responseCallbackJava: 
java.util.function.Consumer[util.Map[TopicPartition, 
DeleteRecordsPartitionResult]] =

Review Comment:
   It seems we don't need to create this object, right? we can pass `response 
=> responseCallback(response.asScala)` directly



##########
server-common/src/main/java/org/apache/kafka/server/common/DeleteRecordsPartitionStatus.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class DeleteRecordsPartitionStatus {
+    private final long requiredOffset;
+    private final DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus;
+    private volatile boolean acksPending;
+
+    public DeleteRecordsPartitionStatus(long requiredOffset,
+                                        
DeleteRecordsResponseData.DeleteRecordsPartitionResult responseStatus) {
+        this.requiredOffset = requiredOffset;
+        this.responseStatus = responseStatus;
+        this.acksPending = false;
+    }
+
+    public boolean acksPending() {
+        return acksPending;
+    }
+
+    public void setAcksPending(boolean acksPending) {
+        this.acksPending = acksPending;
+    }
+
+
+    public DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus() {
+        return responseStatus;
+    }
+
+    public long requiredOffset() {
+        return requiredOffset;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("[acksPending: %b, error: %s, lowWatermark: %d, 
requiredOffset: %d]",
+                acksPending, 
Errors.forCode(responseStatus.errorCode()).toString(), 
responseStatus.lowWatermark(),
+                requiredOffset);
+    }
+
+    /**
+     * A delayed delete records operation that can be created by the replica 
manager and watched
+     * in the delete records operation purgatory
+     */
+    public static class DelayedDeleteRecords extends DelayedOperation {
+        private final Map<TopicPartition, DeleteRecordsPartitionStatus> 
deleteRecordsStatus;
+        private final BiConsumer<TopicPartition, DeleteRecordsPartitionStatus> 
onAcksPending;
+        private final Consumer<Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback;
+        private static final Logger log = 
LoggerFactory.getLogger(DelayedDeleteRecords.class);
+
+        public DelayedDeleteRecords(long delayMs,
+                                    Map<TopicPartition, 
DeleteRecordsPartitionStatus> deleteRecordsStatus,
+                                    //  To maintain compatibility with 
dependency packages, the logic has been moved to the caller.
+                                    BiConsumer<TopicPartition, 
DeleteRecordsPartitionStatus> onAcksPending,
+                                    Consumer<Map<TopicPartition,
+                                            
DeleteRecordsResponseData.DeleteRecordsPartitionResult>> responseCallback) {
+            super(delayMs);
+            this.onAcksPending = onAcksPending;
+            this.deleteRecordsStatus = new 
ConcurrentHashMap<>(deleteRecordsStatus);
+            this.responseCallback = responseCallback;
+            // first update the acks pending variable according to the error 
code
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.responseStatus().errorCode() == Errors.NONE.code()) 
{
+                    // Timeout error state will be cleared when required acks 
are received
+                    status.setAcksPending(true);
+                    
status.responseStatus().setErrorCode(Errors.REQUEST_TIMED_OUT.code());
+                } else {
+                    status.setAcksPending(false);
+                }
+            });
+        }
+
+        /**
+         * The delayed delete records operation can be completed if every 
partition specified in the request satisfied one of the following:
+         *
+         * 1) There was an error while checking if all replicas have caught up 
to the deleteRecordsOffset: set an error in response
+         * 2) The low watermark of the partition has caught up to the 
deleteRecordsOffset. set the low watermark in response
+         *
+         */
+        @Override
+        public boolean tryComplete() {
+            //  check for each partition if it still has pending acks
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                log.trace("Checking delete records satisfaction for {}, 
current status {}", topicPartition, status);
+                //  skip those partitions that have already been satisfied
+                if (status.acksPending()) {
+                    onAcksPending.accept(topicPartition, status);
+                }
+            });
+            //  check if every partition has satisfied at least one of case A 
or B
+            return 
deleteRecordsStatus.values().stream().noneMatch(DeleteRecordsPartitionStatus::acksPending)
 && forceComplete();
+        }
+
+        @Override
+        public void onExpiration() {
+            deleteRecordsStatus.forEach((topicPartition, status) -> {
+                if (status.acksPending()) {
+                    
DelayedDeleteRecordsMetrics.recordExpiration(topicPartition);
+                }
+            });
+        }
+
+        /**
+         * Upon completion, return the current response status along with the 
error code per partition
+         */
+        @Override
+        public void onComplete() {
+            Map<TopicPartition, 
DeleteRecordsResponseData.DeleteRecordsPartitionResult> responseStatus =
+                    new HashMap<>();
+            deleteRecordsStatus.forEach((k, status) -> {
+                responseStatus.put(k, status.responseStatus());
+            });
+            responseCallback.accept(responseStatus);
+        }
+    }
+
+    private static class DelayedDeleteRecordsMetrics {
+        private static final KafkaMetricsGroup METRICS_GROUP = new 
KafkaMetricsGroup(DelayedDeleteRecordsMetrics.class);

Review Comment:
   this break the compatibility of metrics name, so please use 
"kafka.server.DelayedDeleteRecordsMetrics" instead



##########
server-common/src/main/java/org/apache/kafka/server/common/DeleteRecordsPartitionStatus.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class DeleteRecordsPartitionStatus {

Review Comment:
   the main class should be `DelayedDeleteRecords` rather than 
`DeleteRecordsPartitionStatus`



##########
server-common/src/main/java/org/apache/kafka/server/common/DeleteRecordsPartitionStatus.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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;
+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.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class DeleteRecordsPartitionStatus {
+    private final long requiredOffset;
+    private final DeleteRecordsResponseData.DeleteRecordsPartitionResult 
responseStatus;
+    private volatile boolean acksPending;
+
+    public DeleteRecordsPartitionStatus(long requiredOffset,
+                                        
DeleteRecordsResponseData.DeleteRecordsPartitionResult responseStatus) {

Review Comment:
   could you please import 
`DeleteRecordsResponseData.DeleteRecordsPartitionResult`?



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