You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "nizhikov (via GitHub)" <gi...@apache.org> on 2023/02/19 15:35:09 UTC

[GitHub] [kafka] nizhikov opened a new pull request, #13278: KAFKA-14591 DeleteRecordsCommand moved to java

nizhikov opened a new pull request, #13278:
URL: https://github.com/apache/kafka/pull/13278

   This PR contains changes to move `DeleteRecordsCommand` to java code
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1267084525


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.CoreUtils;
+import org.apache.kafka.server.util.Json;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) {
+        try {
+            JsonNode js = Json.tryParseFull(jsonData).node();
+
+            int version = EARLIEST_VERSION;
+
+            if (js.has("version"))
+                version = js.get("version").asInt();
+
+            return parseJsonData(version, js);
+        } catch (JsonProcessingException e) {
+            throw new AdminOperationException("The input string is not a valid JSON");
+        }
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonNode js) throws JsonMappingException {

Review Comment:
   Refactored to use JsonValue when possible.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266988216


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file",
+            "--command-config", "/another/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testWrongVersion() {
+        assertThrowsAdminOperationException("{\"version\":\"string\"}");
+        assertThrowsAdminOperationException("{\"version\":2}");
+    }
+
+    @Test
+    public void testWrongPartitions() {
+        assertThrowsAdminOperationException("{\"version\":1}");
+        assertThrowsAdminOperationException("{\"partitions\":2}");
+        assertThrowsAdminOperationException("{\"partitions\":{}}");
+        assertThrowsAdminOperationException("{\"partitions\":[{}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}");

Review Comment:
   Extra field will be ignored. 



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file",
+            "--command-config", "/another/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testWrongVersion() {
+        assertThrowsAdminOperationException("{\"version\":\"string\"}");
+        assertThrowsAdminOperationException("{\"version\":2}");
+    }
+
+    @Test
+    public void testWrongPartitions() {
+        assertThrowsAdminOperationException("{\"version\":1}");
+        assertThrowsAdminOperationException("{\"partitions\":2}");
+        assertThrowsAdminOperationException("{\"partitions\":{}}");
+        assertThrowsAdminOperationException("{\"partitions\":[{}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}");

Review Comment:
   Extra fields will be ignored. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1258230760


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -16,18 +16,101 @@
  */
 package org.apache.kafka.tools;
 
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
 import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.IOException;
 import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.KRAFT)

Review Comment:
   Why do we need this one?



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -16,18 +16,101 @@
  */
 package org.apache.kafka.tools;
 
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
 import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.IOException;
 import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.KRAFT)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest(clusterType = Type.ZK)

Review Comment:
   It would be good to test both ZK (1 brk) and CO_KRAFT (1 brk, 1 ctr).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1270532744


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   It seems tool semantic not changed with `Map<TopicPartition, List<Long>>`.
   So current changes conform both semantic requirements and your suggestion.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1267093982


##########
server-common/src/main/java/org/apache/kafka/server/util/CoreUtils.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {
+    /**
+     * Returns a list of duplicated items
+     */
+    public static <T> Iterable<T> duplicates(Iterable<T> s) {

Review Comment:
   > I wonder if we should return Set instead of Iterable
   
   Changes to return Set.
   
   > so maybe we can put this into tools?
   
   CoreUtils, CoreUtilsTest move to tools.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266973638


##########
server-common/src/main/java/org/apache/kafka/server/common/AdminCommandFailedException.java:
##########
@@ -15,9 +15,18 @@
  * limitations under the License.
  */
 
-package kafka.admin
+package org.apache.kafka.server.common;
 
-class AdminOperationException(val error: String, cause: Throwable) extends RuntimeException(error, cause) {
-  def this(error: Throwable) = this(error.getMessage, error)
-  def this(msg: String) = this(msg, null)
-}
\ No newline at end of file
+public class AdminCommandFailedException extends RuntimeException {
+    public AdminCommandFailedException(String message) {
+        super(message);
+    }
+
+    public AdminCommandFailedException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public AdminCommandFailedException() {

Review Comment:
   No, removed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266996537


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {

Review Comment:
   > Is this similar in the original tool?
   
   Yes.
   
   `--offset-json-file` read by `Utils.readFileAsString` which converts all `* extens IOExceptions` exceptions to `IOException`:
   
   ```
       public static String readFileAsString(String path) throws IOException {
           try {
               byte[] allBytes = Files.readAllBytes(Paths.get(path));
               return new String(allBytes, StandardCharsets.UTF_8);
           } catch (IOException ex) {
               throw new IOException("Unable to read file " + path, ex);
           }
       }
   ```
   
   and `--command-config` file read by `Utils.loadProps` which throw `NoSuchFileException` from 
   
   ```
           if (filename != null) {
               try (InputStream propStream = Files.newInputStream(Paths.get(filename))) {
                   props.load(propStream);
               }
           } else {
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268278161


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file",
+            "--command-config", "/another/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testWrongVersion() {
+        assertThrowsAdminOperationException("{\"version\":\"string\"}");
+        assertThrowsAdminOperationException("{\"version\":2}");
+    }
+
+    @Test
+    public void testWrongPartitions() {
+        assertThrowsAdminOperationException("{\"version\":1}");
+        assertThrowsAdminOperationException("{\"partitions\":2}");
+        assertThrowsAdminOperationException("{\"partitions\":{}}");
+        assertThrowsAdminOperationException("{\"partitions\":[{}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}");

Review Comment:
   Success parsing test case extended like this:
   
   ```
       @Test
       public void testParse() throws Exception {
           Collection<DeleteRecordsCommand.Tuple<TopicPartition, Long>> res = DeleteRecordsCommand.parseOffsetJsonStringWithoutDedup(
               "{\"partitions\":[" +
                   "{\"topic\":\"t\", \"partition\":0, \"offset\":0}," +
                   "{\"topic\":\"t\", \"partition\":1, \"offset\":1, \"ignored\":\"field\"}," +
                   "{\"topic\":\"t\", \"partition\":0, \"offset\":2}," +
                   "{\"topic\":\"t\", \"partition\":0, \"offset\":0}" +
               "]}"
           );
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] github-actions[bot] commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1585813165

   This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has  merge conflicts, please update it with the latest from trunk (or appropriate release branch)
   If this PR is no longer valid or desired, please feel free to close it. If no activity occurrs in the next 30 days, it will be automatically closed.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268183325


##########
tools/src/main/java/org/apache/kafka/tools/CoreUtils.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.tools;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {

Review Comment:
   We already have ToolsUtils in server-common, and maybe we should think about moving it to the tools module in a separate PR.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268270865


##########
tools/src/test/java/org/apache/kafka/tools/CoreUtilsTest.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.tools;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertIterableEquals;
+
+public class CoreUtilsTest {
+    @Test
+    public void testDuplicates() {
+        assertIterableEquals(

Review Comment:
   Done



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268293020


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   We have similar clases like 
   * `org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.TaskPairs.Pair`
   * `org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerGenerationPair`
   * `org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerPair`
   
   So may be it will be better to keep generic class and reuse it in other places?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison merged pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison merged PR #13278:
URL: https://github.com/apache/kafka/pull/13278


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1645987767

   @fvaleri @mimaison Thank you for the review and merge!


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1640666685

   @mimaison Thanks for the review. It seems I addressed all your comments. Please, take a look one more time.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268183773


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   +1 on this



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1270529100


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   I was just asking, if there's a good reason to keep Tuple, we should keep it and not change the tool's semantics.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1267096601


##########
server-common/src/main/java/org/apache/kafka/server/util/CoreUtils.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {
+    /**
+     * Returns a list of duplicated items
+     */
+    public static <T> Iterable<T> duplicates(Iterable<T> s) {
+        return StreamSupport.stream(s.spliterator(), false)

Review Comment:
   Nice proposal. Thanks. Fixed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266976239


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {

Review Comment:
   Renamed to `testCommand`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1629087276

   > The JSON parsing logic is shared between multiple tools, including this one, so it seems a nice optimization to have.
   
   Sure. We must keep codebase consistent. 
   Anyway, in the moment my PR seems to be complete, isn't it?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1639865393

   #13585 has been merged. Can you please rebase and integrate with this PR?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268252459


##########
tools/src/main/java/org/apache/kafka/tools/CoreUtils.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.tools;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {

Review Comment:
   Renamed



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1642362522

   @mimaison all your comments addressed except the one with the Tuple. Please, review.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1267015295


##########
server-common/src/test/java/org/apache/kafka/server/util/CoreUtilsTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.util;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CoreUtilsTest {
+    @Test
+    public void testDuplicates() {
+        assertIterableEquals(

Review Comment:
   Fixed



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1628597321

   Hello, @fvaleri 
   
   Thanks for the feedback.
   
   > Can you please rebase and add a unit test for the various options, like we have for the other tools?
   
   Yes. I merged trunk to the PR and added tests for rewritten command.
   Please, take a look into `DeleteRecordsCommandTest`.
   
   > Have you considered using the Jackson wrapper [proposed here](https://issues.apache.org/jira/browse/KAFKA-14737)?
   
   I'm not aware of this work.
   It seems right now command works like expected :)


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1629146547

   Yep, LGTM. Just waiting for #13585.
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266975358


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.CoreUtils;
+import org.apache.kafka.server.util.Json;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) {
+        try {
+            JsonNode js = Json.tryParseFull(jsonData).node();
+
+            int version = EARLIEST_VERSION;
+
+            if (js.has("version"))
+                version = js.get("version").asInt();
+
+            return parseJsonData(version, js);
+        } catch (JsonProcessingException e) {
+            throw new AdminOperationException("The input string is not a valid JSON");
+        }
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonNode js) throws JsonMappingException {
+        if (version == 1) {
+            JsonNode partitions = js.get("partitions");
+
+            if (partitions == null || !partitions.isArray())
+                throw new AdminOperationException("Missing partitions field");
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            for (JsonNode partitionJs : partitions) {
+                String topic = getOrThrow(partitionJs, "topic").asText();
+                int partition = getOrThrow(partitionJs, "partition").asInt();
+                long offset = getOrThrow(partitionJs, "offset").asLong();
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute0(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute0(Admin adminClient, String offsetJsonString, PrintStream out) {

Review Comment:
   Renamed



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268286710


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   Regular Map implementations can have only one key -> value mapping.
   But `parseOffsetJsonStringWithoutDedup` collects data with possible duplicates.
   
   I can rework command logic and throw on first `TopicPartition` duplicate, but, it seems out of scope of "just rewrite command in java without changing anything".
   
   Do we have some multi map implementation that can be used here? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1640020567

   @fvaleri Done. New tests passed locally.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1258340747


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -16,18 +16,101 @@
  */
 package org.apache.kafka.tools;
 
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
 import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.IOException;
 import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.KRAFT)

Review Comment:
   No. It seems test works as expected without this annotation. Fixed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1258346596


##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -16,18 +16,101 @@
  */
 package org.apache.kafka.tools;
 
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
 import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.IOException;
 import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.KRAFT)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest(clusterType = Type.ZK)

Review Comment:
   Set `@ClusterTestDefaults(clusterType = Type.ALL)` to test command for all cluster types.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1641386848

   Tests failures unrelated.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1266857754


##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.CoreUtils;
+import org.apache.kafka.server.util.Json;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) {
+        try {
+            JsonNode js = Json.tryParseFull(jsonData).node();
+
+            int version = EARLIEST_VERSION;
+
+            if (js.has("version"))
+                version = js.get("version").asInt();
+
+            return parseJsonData(version, js);
+        } catch (JsonProcessingException e) {
+            throw new AdminOperationException("The input string is not a valid JSON");
+        }
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonNode js) throws JsonMappingException {
+        if (version == 1) {
+            JsonNode partitions = js.get("partitions");
+
+            if (partitions == null || !partitions.isArray())
+                throw new AdminOperationException("Missing partitions field");
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            for (JsonNode partitionJs : partitions) {
+                String topic = getOrThrow(partitionJs, "topic").asText();
+                int partition = getOrThrow(partitionJs, "partition").asInt();
+                long offset = getOrThrow(partitionJs, "offset").asLong();
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute0(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute0(Admin adminClient, String offsetJsonString, PrintStream out) {

Review Comment:
   Could this just be `execute()`?



##########
server-common/src/main/java/org/apache/kafka/server/common/AdminCommandFailedException.java:
##########
@@ -15,9 +15,18 @@
  * limitations under the License.
  */
 
-package kafka.admin
+package org.apache.kafka.server.common;
 
-class AdminOperationException(val error: String, cause: Throwable) extends RuntimeException(error, cause) {
-  def this(error: Throwable) = this(error.getMessage, error)
-  def this(msg: String) = this(msg, null)
-}
\ No newline at end of file
+public class AdminCommandFailedException extends RuntimeException {
+    public AdminCommandFailedException(String message) {
+        super(message);
+    }
+
+    public AdminCommandFailedException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public AdminCommandFailedException() {

Review Comment:
   Do we need this constructor?



##########
checkstyle/import-control.xml:
##########
@@ -295,6 +295,7 @@
 
   <subpackage name="tools">
     <allow pkg="org.apache.kafka.common"/>
+    <allow pkg="org.apache.kafka.server.common" />

Review Comment:
   We already allow that package just below



##########
server-common/src/main/java/org/apache/kafka/server/util/CoreUtils.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {
+    /**
+     * Returns a list of duplicated items
+     */
+    public static <T> Iterable<T> duplicates(Iterable<T> s) {
+        return StreamSupport.stream(s.spliterator(), false)

Review Comment:
   I think it may be one of these cases where a non-stream algorithm might be more readable. Something like:
   ```
   public static <T> Set<T> duplicates2(List<T> set) {
       Set<T> elements = new HashSet<>();
       return set.stream()
               .filter(n -> !elements.add(n))
               .collect(Collectors.toSet());
   }
   ```



##########
server-common/src/main/java/org/apache/kafka/server/util/CoreUtils.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {
+    /**
+     * Returns a list of duplicated items
+     */
+    public static <T> Iterable<T> duplicates(Iterable<T> s) {

Review Comment:
   I wonder if we should return `Set` instead of `Iterable`. The Java `Iterable` is a lot less flexible than its Scala counterpart.



##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.CoreUtils;
+import org.apache.kafka.server.util.Json;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) {
+        try {
+            JsonNode js = Json.tryParseFull(jsonData).node();
+
+            int version = EARLIEST_VERSION;
+
+            if (js.has("version"))
+                version = js.get("version").asInt();
+
+            return parseJsonData(version, js);
+        } catch (JsonProcessingException e) {
+            throw new AdminOperationException("The input string is not a valid JSON");
+        }
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonNode js) throws JsonMappingException {

Review Comment:
   We now have `JsonValue` in `server-common` so can we use that (if it makes sense) like the original logic.



##########
server-common/src/main/java/org/apache/kafka/server/util/CoreUtils.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {
+    /**
+     * Returns a list of duplicated items
+     */
+    public static <T> Iterable<T> duplicates(Iterable<T> s) {

Review Comment:
   Also not sure why this goes into the `server-common` module. The original `duplicates()` method seems to only be called by tools, so maybe we can put this into `tools`?



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {

Review Comment:
   It's strange that the other test above has a different exception for a similar condition (file does not exist). Is this similar in the original tool?



##########
server-common/src/test/java/org/apache/kafka/server/util/CoreUtilsTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.util;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CoreUtilsTest {
+    @Test
+    public void testDuplicates() {
+        assertIterableEquals(

Review Comment:
   The expected value should be the first argument. Same below



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file",
+            "--command-config", "/another/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testWrongVersion() {
+        assertThrowsAdminOperationException("{\"version\":\"string\"}");
+        assertThrowsAdminOperationException("{\"version\":2}");
+    }
+
+    @Test
+    public void testWrongPartitions() {
+        assertThrowsAdminOperationException("{\"version\":1}");
+        assertThrowsAdminOperationException("{\"partitions\":2}");
+        assertThrowsAdminOperationException("{\"partitions\":{}}");
+        assertThrowsAdminOperationException("{\"partitions\":[{}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}");

Review Comment:
   What happens if there are extra fields?



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {

Review Comment:
   Why is it called `testCommandZk`? We set `clusterType` to `ALL` above



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1268090551


##########
tools/src/main/java/org/apache/kafka/tools/CoreUtils.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.tools;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * General helper functions!
+ *
+ * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in
+ * the standard library etc.
+ *
+ * If you are making a new helper function and want to add it to this class please ensure the following:
+ * 1. It has documentation
+ * 2. It is the most general possible utility, not just the thing you needed in one particular place
+ * 3. You have tests for it if it is nontrivial in any way
+ */
+public class CoreUtils {

Review Comment:
   Maybe `ToolsUtils` would be a better name?



##########
tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL)
+@Tag("integration")
+public class DeleteRecordsCommandTest {
+
+    private final ClusterInstance cluster;
+    public DeleteRecordsCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    @ClusterTest
+    public void testCommandZk() throws Exception {
+        Properties adminProps = new Properties();
+
+        adminProps.put(AdminClientConfig.RETRIES_CONFIG, 1);
+
+        try (Admin admin = cluster.createAdminClient(adminProps)) {
+            assertThrows(
+                AdminCommandFailedException.class,
+                () -> DeleteRecordsCommand.execute0(admin, "{\"partitions\":[" +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}," +
+                    "{\"topic\":\"t\", \"partition\":0, \"offset\":1}]" +
+                    "}", System.out),
+                "Offset json file contains duplicate topic partitions: t-0"
+            );
+
+            admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
+
+            Properties props = new Properties();
+
+            props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+            props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+            props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+
+            try (KafkaProducer<?, String> producer = new KafkaProducer<>(props)) {
+                producer.send(new ProducerRecord<>("t", "1")).get();
+                producer.send(new ProducerRecord<>("t", "2")).get();
+                producer.send(new ProducerRecord<>("t", "3")).get();
+            }
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":0, \"offset\":1}]}",
+                "partition: t-0\tlow_watermark: 1",
+                admin
+            );
+
+            executeAndAssertOutput(
+                "{\"partitions\":[{\"topic\":\"t\", \"partition\":42, \"offset\":42}]}",
+                "partition: t-42\terror",
+                admin
+            );
+        }
+    }
+
+    private static void executeAndAssertOutput(String json, String expOut, Admin admin) {
+        String output =
+            ToolsTestUtils.captureStandardOut(() -> DeleteRecordsCommand.execute0(admin, json, System.out));
+        assertTrue(output.contains(expOut));
+    }
+}
+
+/**
+ * Unit test of {@link DeleteRecordsCommand} tool.
+ */
+class DeleteRecordsCommandUnitTest {
+    @Test
+    public void testOffsetFileNotExists() {
+        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testCommandConfigNotExists() {
+        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
+            "--bootstrap-server", "localhost:9092",
+            "--offset-json-file", "/not/existing/file",
+            "--command-config", "/another/not/existing/file"
+        }));
+    }
+
+    @Test
+    public void testWrongVersion() {
+        assertThrowsAdminOperationException("{\"version\":\"string\"}");
+        assertThrowsAdminOperationException("{\"version\":2}");
+    }
+
+    @Test
+    public void testWrongPartitions() {
+        assertThrowsAdminOperationException("{\"version\":1}");
+        assertThrowsAdminOperationException("{\"partitions\":2}");
+        assertThrowsAdminOperationException("{\"partitions\":{}}");
+        assertThrowsAdminOperationException("{\"partitions\":[{}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}");
+        assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}");

Review Comment:
   Should we have a test for it?



##########
tools/src/test/java/org/apache/kafka/tools/CoreUtilsTest.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.tools;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertIterableEquals;
+
+public class CoreUtilsTest {
+    @Test
+    public void testDuplicates() {
+        assertIterableEquals(

Review Comment:
   Can we use `assertEquals()` now that `duplicates()` returns a `Set`?



##########
tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.tools;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.DeleteRecordsResult;
+import org.apache.kafka.clients.admin.RecordsToDelete;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.Json;
+import org.apache.kafka.server.util.json.DecodeJson;
+import org.apache.kafka.server.util.json.JsonObject;
+import org.apache.kafka.server.util.json.JsonValue;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * A command for delete records of the given partitions down to the specified offset.
+ */
+public class DeleteRecordsCommand {
+    private static final int EARLIEST_VERSION = 1;
+
+    private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
+
+    private static final DecodeJson.DecodeLong LONG = new DecodeJson.DecodeLong();
+
+    private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
+
+    public static void main(String[] args) throws Exception {
+        execute(args, System.out);
+    }
+
+    static Collection<Tuple<TopicPartition, Long>> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
+        JsonValue js = Json.parseFull(jsonData)
+            .orElseThrow(() -> new AdminOperationException("The input string is not a valid JSON"));
+
+        Optional<JsonValue> version = js.asJsonObject().get("version");
+
+        return parseJsonData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js);
+    }
+
+    private static Collection<Tuple<TopicPartition, Long>> parseJsonData(int version, JsonValue js) throws JsonMappingException {
+        if (version == 1) {
+            JsonValue partitions = js.asJsonObject().get("partitions")
+                .orElseThrow(() -> new AdminOperationException("Missing partitions field"));
+
+            Collection<Tuple<TopicPartition, Long>> res = new ArrayList<>();
+
+            Iterator<JsonValue> iterator = partitions.asJsonArray().iterator();
+
+            while (iterator.hasNext()) {
+                JsonObject partitionJs = iterator.next().asJsonObject();
+
+                String topic = partitionJs.apply("topic").to(STRING);
+                int partition = partitionJs.apply("partition").to(INT);
+                long offset = partitionJs.apply("offset").to(LONG);
+
+                res.add(new Tuple<>(new TopicPartition(topic, partition), offset));
+            }
+
+            return res;
+        }
+
+        throw new AdminOperationException("Not supported version field value " + version);
+    }
+
+    public static void execute(String[] args, PrintStream out) throws IOException {
+        DeleteRecordsCommandOptions opts = new DeleteRecordsCommandOptions(args);
+
+        try (Admin adminClient = createAdminClient(opts)) {
+            execute(adminClient, Utils.readFileAsString(opts.options.valueOf(opts.offsetJsonFileOpt)), out);
+        }
+    }
+
+    static void execute(Admin adminClient, String offsetJsonString, PrintStream out) throws JsonProcessingException {
+        Collection<Tuple<TopicPartition, Long>> offsetSeq = parseOffsetJsonStringWithoutDedup(offsetJsonString);
+
+        Set<TopicPartition> duplicatePartitions =
+            CoreUtils.duplicates(offsetSeq.stream().map(Tuple::v1).collect(Collectors.toList()));
+
+        if (!duplicatePartitions.isEmpty()) {
+            StringJoiner duplicates = new StringJoiner(",");
+            duplicatePartitions.forEach(tp -> duplicates.add(tp.toString()));
+            throw new AdminCommandFailedException(
+                String.format("Offset json file contains duplicate topic partitions: %s", duplicates)
+            );
+        }
+
+        Map<TopicPartition, RecordsToDelete> recordsToDelete = offsetSeq.stream()
+            .map(tuple -> new Tuple<>(tuple.v1, RecordsToDelete.beforeOffset(tuple.v2)))
+            .collect(Collectors.toMap(Tuple::v1, Tuple::v2));
+
+        out.println("Executing records delete operation");
+        DeleteRecordsResult deleteRecordsResult = adminClient.deleteRecords(recordsToDelete);
+        out.println("Records delete operation completed:");
+
+        deleteRecordsResult.lowWatermarks().forEach((tp, partitionResult) -> {
+            try {
+                out.printf("partition: %s\tlow_watermark: %s%n", tp, partitionResult.get().lowWatermark());
+            } catch (InterruptedException | ExecutionException e) {
+                out.printf("partition: %s\terror: %s%n", tp, e.getMessage());
+            }
+        });
+    }
+
+    private static Admin createAdminClient(DeleteRecordsCommandOptions opts) throws IOException {
+        Properties props = opts.options.has(opts.commandConfigOpt)
+            ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+            : new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
+        return Admin.create(props);
+    }
+
+    private static class DeleteRecordsCommandOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> offsetJsonFileOpt;
+        private final OptionSpec<String> commandConfigOpt;
+
+        public DeleteRecordsCommandOptions(String[] args) {
+            super(args);
+
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server to connect to.")
+                .withRequiredArg()
+                .describedAs("server(s) to use for bootstrapping")
+                .ofType(String.class);
+
+            offsetJsonFileOpt = parser.accepts("offset-json-file", "REQUIRED: The JSON file with offset per partition. " +
+                    "The format to use is:\n" +
+                    "{\"partitions\":\n  [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}")
+                .withRequiredArg()
+                .describedAs("Offset json file path")
+                .ofType(String.class);
+
+            commandConfigOpt = parser.accepts("command-config", "A property file containing configs to be passed to Admin Client.")
+                .withRequiredArg()
+                .describedAs("command config property file path")
+                .ofType(String.class);
+
+            options = parser.parse(args);
+
+            CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to delete records of the given partitions down to the specified offset.");
+
+            CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt, offsetJsonFileOpt);
+        }
+    }
+
+    public static final class Tuple<V1, V2> {

Review Comment:
   Do we really need this class? Why can't we use something like `Map`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on a diff in pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on code in PR #13278:
URL: https://github.com/apache/kafka/pull/13278#discussion_r1267000462


##########
checkstyle/import-control.xml:
##########
@@ -295,6 +295,7 @@
 
   <subpackage name="tools">
     <allow pkg="org.apache.kafka.common"/>
+    <allow pkg="org.apache.kafka.server.common" />

Review Comment:
   Fixed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nizhikov commented on pull request #13278: KAFKA-14591 DeleteRecordsCommand moved to tools

Posted by "nizhikov (via GitHub)" <gi...@apache.org>.
nizhikov commented on PR #13278:
URL: https://github.com/apache/kafka/pull/13278#issuecomment-1643445734

   @mimaison I reworked `parseOffsetJsonStringWithoutDedup` to return `Map<TopicPartition, List<Long>>`.
   Now, Tuple class eliminated from PR. Please, review.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org