steveloughran commented on code in PR #7197:
URL: https://github.com/apache/hadoop/pull/7197#discussion_r1865867760
##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestBulkDeleteCommand.java:
##########
@@ -0,0 +1,45 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.LinkedList;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBulkDeleteCommand {
Review Comment:
extends AbstractHadoopTestBase
adds timeouts, names threads, give you a base setup() and teardown to
subclass
##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestBulkDeleteCommand.java:
##########
@@ -0,0 +1,45 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.LinkedList;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBulkDeleteCommand {
+ private static Configuration conf;
+
+ @BeforeClass
+ public static void setup() throws IOException {
+ conf = new Configuration();
+ }
+
+ @Test
+ public void testDefaults() throws IOException {
+ LinkedList<String> options = new LinkedList<>();
+ BulkDeleteCommand bulkDeleteCommand = new BulkDeleteCommand();
+ bulkDeleteCommand.processOptions(options);
+ assertTrue(bulkDeleteCommand.childArgs.isEmpty());
+ }
+
+ @Test
+ public void testArguments() throws IOException, URISyntaxException {
+ BulkDeleteCommand bulkDeleteCommand = new BulkDeleteCommand(conf);
+ LinkedList<String> arguments = new LinkedList<>();
+ String arg1 = "file:///file/name/1";
+ String arg2 = "file:///file/name/2";
+ arguments.add(arg1);
+ arguments.add(arg2);
+ LinkedList<PathData> pathData =
bulkDeleteCommand.expandArguments(arguments);
+ assertEquals(1, pathData.size());
Review Comment:
use assertJ list commands, as they are really good at printing lists on
failures
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/BulkDeleteCommand.java:
##########
@@ -0,0 +1,84 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
Review Comment:
ordering should be
```
java.*
javax.*
non apache
org.apache
static
```
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java:
##########
@@ -71,6 +71,7 @@ public static void registerCommands(CommandFactory factory) {
factory.registerCommands(SnapshotCommands.class);
factory.registerCommands(XAttrCommands.class);
factory.registerCommands(Concat.class);
+ factory.registerCommands(BulkDeleteCommand.class);
Review Comment:
move To L54 so keep in approximate alphabetical order
##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestBulkDeleteCommand.java:
##########
@@ -0,0 +1,45 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.LinkedList;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestBulkDeleteCommand {
+ private static Configuration conf;
+
+ @BeforeClass
+ public static void setup() throws IOException {
+ conf = new Configuration();
+ }
+
+ @Test
+ public void testDefaults() throws IOException {
+ LinkedList<String> options = new LinkedList<>();
+ BulkDeleteCommand bulkDeleteCommand = new BulkDeleteCommand();
+ bulkDeleteCommand.processOptions(options);
+ assertTrue(bulkDeleteCommand.childArgs.isEmpty());
Review Comment:
prefer assertJ with description. otherwise, add a text message in the assert
to explain the failure
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/BulkDeleteCommand.java:
##########
@@ -0,0 +1,84 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BulkDelete;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class BulkDeleteCommand extends FsCommand {
+ public static void registerCommands(CommandFactory factory) {
+ factory.addClass(BulkDeleteCommand.class, "-bulkDelete");
+ }
+
+ public static final String name = "bulkDelete";
+
+ public static final String READ_FROM_FILE = "readFromFile";
+
+ public static final String USAGE = "-[ " + READ_FROM_FILE + "] [<file>]
[<basePath> <paths>]";
+
+ public static final String DESCRIPTION = "Deletes the set of files under
the given path. If a list of paths " +
+ "is provided then the paths are deleted directly. User can also
point to the file where the paths are" +
+ "listed as full object names.";
+
+ private String fileName;
+
+ /*
+ Making the class stateful as the PathData initialization for all args is
not needed
+ */
+ LinkedList<String> childArgs;
+
+ protected BulkDeleteCommand() {
+ this.childArgs = new LinkedList<>();
+ }
+
+ protected BulkDeleteCommand(Configuration conf) {super(conf);}
+
+ @Override
+ protected void processOptions(LinkedList<String> args) throws IOException {
+ CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE);
+ cf.addOptionWithValue(READ_FROM_FILE);
+ cf.parse(args);
+ fileName = cf.getOptValue(READ_FROM_FILE);
+ }
+
+ @Override
+ protected LinkedList<PathData> expandArguments(LinkedList<String> args)
throws IOException {
+ if(fileName == null && args.size() < 2) {
+ throw new IOException("Invalid Number of Arguments. Expected
more");
+ }
+ LinkedList<PathData> pathData = new LinkedList<>();
+ pathData.add(new PathData(args.get(0), getConf()));
+ args.remove(0);
+ this.childArgs = args;
+ return pathData;
+ }
+
+ @Override
+ protected void processArguments(LinkedList<PathData> args) throws
IOException {
+ PathData basePath = args.get(0);
+ out.println("Deleting files under:" + basePath);
+ List<Path> pathList = new ArrayList<>();
+ if(fileName != null) {
+ FileSystem localFile = FileSystem.get(getConf());
+ BufferedReader br = new BufferedReader(new
InputStreamReader(localFile.open(new Path(fileName))));
+ String line;
+ while((line = br.readLine()) != null) {
+ pathList.add(new Path(line));
+ }
+ } else {
+
pathList.addAll(this.childArgs.stream().map(Path::new).collect(Collectors.toList()));
+ }
+ BulkDelete bulkDelete = basePath.fs.createBulkDelete(basePath.path);
+ bulkDelete.bulkDelete(pathList);
Review Comment:
2. get the pagesize and do the bulk delete in pages; log page size @ debug.
1. can you catch and log failures here. just because the other fs commands
don't, doesn't mean we should follow.
2. at the end use duration tracker to log duration, also print the # path
size
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/BulkDeleteCommand.java:
##########
@@ -0,0 +1,84 @@
+package org.apache.hadoop.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BulkDelete;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class BulkDeleteCommand extends FsCommand {
+ public static void registerCommands(CommandFactory factory) {
+ factory.addClass(BulkDeleteCommand.class, "-bulkDelete");
+ }
+
+ public static final String name = "bulkDelete";
+
+ public static final String READ_FROM_FILE = "readFromFile";
+
+ public static final String USAGE = "-[ " + READ_FROM_FILE + "] [<file>]
[<basePath> <paths>]";
+
+ public static final String DESCRIPTION = "Deletes the set of files under
the given path. If a list of paths " +
+ "is provided then the paths are deleted directly. User can also
point to the file where the paths are" +
+ "listed as full object names.";
+
+ private String fileName;
+
+ /*
+ Making the class stateful as the PathData initialization for all args is
not needed
+ */
+ LinkedList<String> childArgs;
+
+ protected BulkDeleteCommand() {
+ this.childArgs = new LinkedList<>();
+ }
+
+ protected BulkDeleteCommand(Configuration conf) {super(conf);}
+
+ @Override
+ protected void processOptions(LinkedList<String> args) throws IOException {
+ CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE);
+ cf.addOptionWithValue(READ_FROM_FILE);
+ cf.parse(args);
+ fileName = cf.getOptValue(READ_FROM_FILE);
+ }
+
+ @Override
+ protected LinkedList<PathData> expandArguments(LinkedList<String> args)
throws IOException {
+ if(fileName == null && args.size() < 2) {
+ throw new IOException("Invalid Number of Arguments. Expected
more");
+ }
+ LinkedList<PathData> pathData = new LinkedList<>();
+ pathData.add(new PathData(args.get(0), getConf()));
+ args.remove(0);
+ this.childArgs = args;
+ return pathData;
+ }
+
+ @Override
+ protected void processArguments(LinkedList<PathData> args) throws
IOException {
+ PathData basePath = args.get(0);
+ out.println("Deleting files under:" + basePath);
+ List<Path> pathList = new ArrayList<>();
+ if(fileName != null) {
+ FileSystem localFile = FileSystem.get(getConf());
+ BufferedReader br = new BufferedReader(new
InputStreamReader(localFile.open(new Path(fileName))));
+ String line;
+ while((line = br.readLine()) != null) {
Review Comment:
skip lines with # too, just in case.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]