|
@@ -26,11 +26,7 @@ import java.nio.file.Files;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
-import java.util.concurrent.ThreadPoolExecutor;
|
|
|
|
-import java.util.concurrent.ArrayBlockingQueue;
|
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
-import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
@@ -38,8 +34,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.PathIsDirectoryException;
|
|
import org.apache.hadoop.fs.PathIsDirectoryException;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
-import org.slf4j.Logger;
|
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
/** Various commands for copy files */
|
|
/** Various commands for copy files */
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@@ -210,28 +204,37 @@ class CopyCommands {
|
|
/**
|
|
/**
|
|
* Copy local files to a remote filesystem
|
|
* Copy local files to a remote filesystem
|
|
*/
|
|
*/
|
|
- public static class Get extends CommandWithDestination {
|
|
|
|
|
|
+ public static class Get extends CopyCommandWithMultiThread {
|
|
public static final String NAME = "get";
|
|
public static final String NAME = "get";
|
|
public static final String USAGE =
|
|
public static final String USAGE =
|
|
- "[-f] [-p] [-ignoreCrc] [-crc] <src> ... <localdst>";
|
|
|
|
|
|
+ "[-f] [-p] [-crc] [-ignoreCrc] [-t <thread count>]"
|
|
|
|
+ + " [-q <thread pool queue size>] <src> ... <localdst>";
|
|
public static final String DESCRIPTION =
|
|
public static final String DESCRIPTION =
|
|
- "Copy files that match the file pattern <src> " +
|
|
|
|
- "to the local name. <src> is kept. When copying multiple " +
|
|
|
|
- "files, the destination must be a directory. Passing " +
|
|
|
|
- "-f overwrites the destination if it already exists and " +
|
|
|
|
- "-p preserves access and modification times, " +
|
|
|
|
- "ownership and the mode.\n";
|
|
|
|
|
|
+ "Copy files that match the file pattern <src> to the local name. "
|
|
|
|
+ + "<src> is kept.\nWhen copying multiple files, the destination"
|
|
|
|
+ + " must be a directory.\nFlags:\n"
|
|
|
|
+ + " -p : Preserves timestamps, ownership and the mode.\n"
|
|
|
|
+ + " -f : Overwrites the destination if it already exists.\n"
|
|
|
|
+ + " -crc : write CRC checksums for the files downloaded.\n"
|
|
|
|
+ + " -ignoreCrc : Skip CRC checks on the file(s) downloaded.\n"
|
|
|
|
+ + " -t <thread count> : Number of threads to be used,"
|
|
|
|
+ + " default is 1.\n"
|
|
|
|
+ + " -q <thread pool queue size> : Thread pool queue size to be"
|
|
|
|
+ + " used, default is 1024.\n";
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- protected void processOptions(LinkedList<String> args)
|
|
|
|
- throws IOException {
|
|
|
|
- CommandFormat cf = new CommandFormat(
|
|
|
|
- 1, Integer.MAX_VALUE, "crc", "ignoreCrc", "p", "f");
|
|
|
|
|
|
+ protected void processOptions(LinkedList<String> args) throws IOException {
|
|
|
|
+ CommandFormat cf =
|
|
|
|
+ new CommandFormat(1, Integer.MAX_VALUE, "crc", "ignoreCrc", "p", "f");
|
|
|
|
+ cf.addOptionWithValue("t");
|
|
|
|
+ cf.addOptionWithValue("q");
|
|
cf.parse(args);
|
|
cf.parse(args);
|
|
setWriteChecksum(cf.getOpt("crc"));
|
|
setWriteChecksum(cf.getOpt("crc"));
|
|
setVerifyChecksum(!cf.getOpt("ignoreCrc"));
|
|
setVerifyChecksum(!cf.getOpt("ignoreCrc"));
|
|
setPreserve(cf.getOpt("p"));
|
|
setPreserve(cf.getOpt("p"));
|
|
setOverwrite(cf.getOpt("f"));
|
|
setOverwrite(cf.getOpt("f"));
|
|
|
|
+ setThreadCount(cf.getOptValue("t"));
|
|
|
|
+ setThreadPoolQueueSize(cf.getOptValue("q"));
|
|
setRecursive(true);
|
|
setRecursive(true);
|
|
getLocalDestination(args);
|
|
getLocalDestination(args);
|
|
}
|
|
}
|
|
@@ -240,21 +243,12 @@ class CopyCommands {
|
|
/**
|
|
/**
|
|
* Copy local files to a remote filesystem
|
|
* Copy local files to a remote filesystem
|
|
*/
|
|
*/
|
|
- public static class Put extends CommandWithDestination {
|
|
|
|
-
|
|
|
|
- public static final Logger LOG = LoggerFactory.getLogger(Put.class);
|
|
|
|
-
|
|
|
|
- private ThreadPoolExecutor executor = null;
|
|
|
|
- private int threadPoolQueueSize = 1024;
|
|
|
|
- private int numThreads = 1;
|
|
|
|
-
|
|
|
|
- private static final int MAX_THREADS =
|
|
|
|
- Runtime.getRuntime().availableProcessors() * 2;
|
|
|
|
|
|
+ public static class Put extends CopyCommandWithMultiThread {
|
|
|
|
|
|
public static final String NAME = "put";
|
|
public static final String NAME = "put";
|
|
public static final String USAGE =
|
|
public static final String USAGE =
|
|
- "[-f] [-p] [-l] [-d] [-t <thread count>] [-q <threadPool queue size>] " +
|
|
|
|
- "<localsrc> ... <dst>";
|
|
|
|
|
|
+ "[-f] [-p] [-l] [-d] [-t <thread count>] [-q <thread pool queue size>]"
|
|
|
|
+ + " <localsrc> ... <dst>";
|
|
public static final String DESCRIPTION =
|
|
public static final String DESCRIPTION =
|
|
"Copy files from the local file system " +
|
|
"Copy files from the local file system " +
|
|
"into fs. Copying fails if the file already " +
|
|
"into fs. Copying fails if the file already " +
|
|
@@ -263,11 +257,11 @@ class CopyCommands {
|
|
" -p : Preserves timestamps, ownership and the mode.\n" +
|
|
" -p : Preserves timestamps, ownership and the mode.\n" +
|
|
" -f : Overwrites the destination if it already exists.\n" +
|
|
" -f : Overwrites the destination if it already exists.\n" +
|
|
" -t <thread count> : Number of threads to be used, default is 1.\n" +
|
|
" -t <thread count> : Number of threads to be used, default is 1.\n" +
|
|
- " -q <threadPool size> : ThreadPool queue size to be used, " +
|
|
|
|
|
|
+ " -q <thread pool queue size> : Thread pool queue size to be used, " +
|
|
"default is 1024.\n" +
|
|
"default is 1024.\n" +
|
|
- " -l : Allow DataNode to lazily persist the file to disk. Forces" +
|
|
|
|
- " replication factor of 1. This flag will result in reduced" +
|
|
|
|
- " durability. Use with care.\n" +
|
|
|
|
|
|
+ " -l : Allow DataNode to lazily persist the file to disk. Forces " +
|
|
|
|
+ "replication factor of 1. This flag will result in reduced " +
|
|
|
|
+ "durability. Use with care.\n" +
|
|
" -d : Skip creation of temporary file(<dst>._COPYING_).\n";
|
|
" -d : Skip creation of temporary file(<dst>._COPYING_).\n";
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -277,7 +271,7 @@ class CopyCommands {
|
|
cf.addOptionWithValue("t");
|
|
cf.addOptionWithValue("t");
|
|
cf.addOptionWithValue("q");
|
|
cf.addOptionWithValue("q");
|
|
cf.parse(args);
|
|
cf.parse(args);
|
|
- setNumberThreads(cf.getOptValue("t"));
|
|
|
|
|
|
+ setThreadCount(cf.getOptValue("t"));
|
|
setThreadPoolQueueSize(cf.getOptValue("q"));
|
|
setThreadPoolQueueSize(cf.getOptValue("q"));
|
|
setOverwrite(cf.getOpt("f"));
|
|
setOverwrite(cf.getOpt("f"));
|
|
setPreserve(cf.getOpt("p"));
|
|
setPreserve(cf.getOpt("p"));
|
|
@@ -308,92 +302,9 @@ class CopyCommands {
|
|
copyStreamToTarget(System.in, getTargetPath(args.get(0)));
|
|
copyStreamToTarget(System.in, getTargetPath(args.get(0)));
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
-
|
|
|
|
- executor = new ThreadPoolExecutor(numThreads, numThreads, 1,
|
|
|
|
- TimeUnit.SECONDS, new ArrayBlockingQueue<>(threadPoolQueueSize),
|
|
|
|
- new ThreadPoolExecutor.CallerRunsPolicy());
|
|
|
|
super.processArguments(args);
|
|
super.processArguments(args);
|
|
-
|
|
|
|
- // issue the command and then wait for it to finish
|
|
|
|
- executor.shutdown();
|
|
|
|
- try {
|
|
|
|
- executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MINUTES);
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
|
- executor.shutdownNow();
|
|
|
|
- displayError(e);
|
|
|
|
- Thread.currentThread().interrupt();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setNumberThreads(String numberThreadsString) {
|
|
|
|
- if (numberThreadsString == null) {
|
|
|
|
- numThreads = 1;
|
|
|
|
- } else {
|
|
|
|
- int parsedValue = Integer.parseInt(numberThreadsString);
|
|
|
|
- if (parsedValue <= 1) {
|
|
|
|
- numThreads = 1;
|
|
|
|
- } else if (parsedValue > MAX_THREADS) {
|
|
|
|
- numThreads = MAX_THREADS;
|
|
|
|
- } else {
|
|
|
|
- numThreads = parsedValue;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setThreadPoolQueueSize(String numThreadPoolQueueSize) {
|
|
|
|
- if (numThreadPoolQueueSize != null) {
|
|
|
|
- int parsedValue = Integer.parseInt(numThreadPoolQueueSize);
|
|
|
|
- if (parsedValue < 1) {
|
|
|
|
- LOG.warn("The value of the thread pool queue size cannot be " +
|
|
|
|
- "less than 1, and the default value is used here. " +
|
|
|
|
- "The default size is 1024.");
|
|
|
|
- threadPoolQueueSize = 1024;
|
|
|
|
- } else {
|
|
|
|
- threadPoolQueueSize = parsedValue;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- protected int getThreadPoolQueueSize() {
|
|
|
|
- return threadPoolQueueSize;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void copyFile(PathData src, PathData target) throws IOException {
|
|
|
|
- if (isPathRecursable(src)) {
|
|
|
|
- throw new PathIsDirectoryException(src.toString());
|
|
|
|
- }
|
|
|
|
- super.copyFileToTarget(src, target);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- protected void copyFileToTarget(PathData src, PathData target)
|
|
|
|
- throws IOException {
|
|
|
|
- // if number of thread is 1, mimic put and avoid threading overhead
|
|
|
|
- if (numThreads == 1) {
|
|
|
|
- copyFile(src, target);
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- Runnable task = () -> {
|
|
|
|
- try {
|
|
|
|
- copyFile(src, target);
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- displayError(e);
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
- executor.submit(task);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public int getNumThreads() {
|
|
|
|
- return numThreads;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public ThreadPoolExecutor getExecutor() {
|
|
|
|
- return executor;
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
|
|
|
|
public static class CopyFromLocal extends Put {
|
|
public static class CopyFromLocal extends Put {
|