|
@@ -29,13 +29,17 @@ import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
|
|
import org.apache.hadoop.tools.util.DistCpUtils;
|
|
|
+import org.apache.hadoop.tools.util.ProducerConsumer;
|
|
|
+import org.apache.hadoop.tools.util.WorkReport;
|
|
|
+import org.apache.hadoop.tools.util.WorkRequest;
|
|
|
+import org.apache.hadoop.tools.util.WorkRequestProcessor;
|
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
import java.io.*;
|
|
|
-import java.util.Stack;
|
|
|
+import java.util.ArrayList;
|
|
|
|
|
|
import static org.apache.hadoop.tools.DistCpConstants
|
|
|
.HDFS_RESERVED_RAW_DIRECTORY_NAME;
|
|
@@ -50,7 +54,10 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
private static final Log LOG = LogFactory.getLog(SimpleCopyListing.class);
|
|
|
|
|
|
private long totalPaths = 0;
|
|
|
+ private long totalDirs = 0;
|
|
|
private long totalBytesToCopy = 0;
|
|
|
+ private int numListstatusThreads = 1;
|
|
|
+ private final int maxRetries = 3;
|
|
|
|
|
|
/**
|
|
|
* Protected constructor, to initialize configuration.
|
|
@@ -61,6 +68,16 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
*/
|
|
|
protected SimpleCopyListing(Configuration configuration, Credentials credentials) {
|
|
|
super(configuration, credentials);
|
|
|
+ numListstatusThreads = getConf().getInt(
|
|
|
+ DistCpConstants.CONF_LABEL_LISTSTATUS_THREADS,
|
|
|
+ DistCpConstants.DEFAULT_LISTSTATUS_THREADS);
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ protected SimpleCopyListing(Configuration configuration, Credentials credentials,
|
|
|
+ int numListstatusThreads) {
|
|
|
+ super(configuration, credentials);
|
|
|
+ this.numListstatusThreads = numListstatusThreads;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -160,6 +177,10 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
@VisibleForTesting
|
|
|
public void doBuildListing(SequenceFile.Writer fileListWriter,
|
|
|
DistCpOptions options) throws IOException {
|
|
|
+ if (options.getNumListstatusThreads() > 0) {
|
|
|
+ numListstatusThreads = options.getNumListstatusThreads();
|
|
|
+ }
|
|
|
+
|
|
|
try {
|
|
|
for (Path path: options.getSourcePaths()) {
|
|
|
FileSystem sourceFS = path.getFileSystem(getConf());
|
|
@@ -181,6 +202,7 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
sourcePathRoot, options);
|
|
|
}
|
|
|
if (explore) {
|
|
|
+ ArrayList<FileStatus> sourceDirs = new ArrayList<FileStatus>();
|
|
|
for (FileStatus sourceStatus: sourceFiles) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Recording source-path: " + sourceStatus.getPath() + " for copy.");
|
|
@@ -195,15 +217,18 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
|
|
|
if (sourceStatus.isDirectory()) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Traversing source dir: " + sourceStatus.getPath());
|
|
|
+ LOG.debug("Adding source dir for traverse: " + sourceStatus.getPath());
|
|
|
}
|
|
|
- traverseDirectory(fileListWriter, sourceFS, sourceStatus,
|
|
|
- sourcePathRoot, options);
|
|
|
+ sourceDirs.add(sourceStatus);
|
|
|
}
|
|
|
}
|
|
|
+ traverseDirectory(fileListWriter, sourceFS, sourceDirs,
|
|
|
+ sourcePathRoot, options);
|
|
|
}
|
|
|
}
|
|
|
fileListWriter.close();
|
|
|
+ printStats();
|
|
|
+ LOG.info("Build file listing completed.");
|
|
|
fileListWriter = null;
|
|
|
} finally {
|
|
|
IOUtils.cleanup(LOG, fileListWriter);
|
|
@@ -275,43 +300,135 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE));
|
|
|
}
|
|
|
|
|
|
- private static FileStatus[] getChildren(FileSystem fileSystem,
|
|
|
- FileStatus parent) throws IOException {
|
|
|
- return fileSystem.listStatus(parent.getPath());
|
|
|
+ /*
|
|
|
+ * Private class to implement WorkRequestProcessor interface. It processes
|
|
|
+ * each directory (represented by FileStatus item) and returns a list of all
|
|
|
+ * file-system objects in that directory (files and directories). In case of
|
|
|
+ * retriable exceptions it increments retry counter and returns the same
|
|
|
+ * directory for later retry.
|
|
|
+ */
|
|
|
+ private static class FileStatusProcessor
|
|
|
+ implements WorkRequestProcessor<FileStatus, FileStatus[]> {
|
|
|
+ private FileSystem fileSystem;
|
|
|
+
|
|
|
+ public FileStatusProcessor(FileSystem fileSystem) {
|
|
|
+ this.fileSystem = fileSystem;
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Processor for FileSystem.listStatus().
|
|
|
+ *
|
|
|
+ * @param workRequest Input work item that contains FileStatus item which
|
|
|
+ * is a parent directory we want to list.
|
|
|
+ * @return Outputs WorkReport<FileStatus[]> with a list of objects in the
|
|
|
+ * directory (array of objects, empty if parent directory is
|
|
|
+ * empty). In case of intermittent exception we increment retry
|
|
|
+ * counter and return the list containing the parent directory).
|
|
|
+ */
|
|
|
+ public WorkReport<FileStatus[]> processItem(
|
|
|
+ WorkRequest<FileStatus> workRequest) {
|
|
|
+ FileStatus parent = workRequest.getItem();
|
|
|
+ int retry = workRequest.getRetry();
|
|
|
+ WorkReport<FileStatus[]> result = null;
|
|
|
+ try {
|
|
|
+ if (retry > 0) {
|
|
|
+ int sleepSeconds = 2;
|
|
|
+ for (int i = 1; i < retry; i++) {
|
|
|
+ sleepSeconds *= 2;
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ Thread.sleep(1000 * sleepSeconds);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ LOG.debug("Interrupted while sleeping in exponential backoff.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ result = new WorkReport<FileStatus[]>(
|
|
|
+ fileSystem.listStatus(parent.getPath()), 0, true);
|
|
|
+ } catch (FileNotFoundException fnf) {
|
|
|
+ LOG.error("FileNotFoundException exception in listStatus: " +
|
|
|
+ fnf.getMessage());
|
|
|
+ result = new WorkReport<FileStatus[]>(new FileStatus[0], 0, true, fnf);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("Exception in listStatus. Will send for retry.");
|
|
|
+ FileStatus[] parentList = new FileStatus[1];
|
|
|
+ parentList[0] = parent;
|
|
|
+ result = new WorkReport<FileStatus[]>(parentList, retry + 1, false, e);
|
|
|
+ }
|
|
|
+ return result;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void printStats() {
|
|
|
+ LOG.info("Paths (files+dirs) cnt = " + totalPaths +
|
|
|
+ "; dirCnt = " + totalDirs);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void maybePrintStats() {
|
|
|
+ if (totalPaths % 100000 == 0) {
|
|
|
+ printStats();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private void traverseDirectory(SequenceFile.Writer fileListWriter,
|
|
|
FileSystem sourceFS,
|
|
|
- FileStatus sourceStatus,
|
|
|
+ ArrayList<FileStatus> sourceDirs,
|
|
|
Path sourcePathRoot,
|
|
|
DistCpOptions options)
|
|
|
throws IOException {
|
|
|
final boolean preserveAcls = options.shouldPreserve(FileAttribute.ACL);
|
|
|
final boolean preserveXAttrs = options.shouldPreserve(FileAttribute.XATTR);
|
|
|
final boolean preserveRawXattrs = options.shouldPreserveRawXattrs();
|
|
|
- Stack<FileStatus> pathStack = new Stack<FileStatus>();
|
|
|
- pathStack.push(sourceStatus);
|
|
|
|
|
|
- while (!pathStack.isEmpty()) {
|
|
|
- for (FileStatus child: getChildren(sourceFS, pathStack.pop())) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Recording source-path: " + child.getPath() + " for copy.");
|
|
|
- }
|
|
|
- CopyListingFileStatus childCopyListingStatus =
|
|
|
- DistCpUtils.toCopyListingFileStatus(sourceFS, child,
|
|
|
- preserveAcls && child.isDirectory(),
|
|
|
- preserveXAttrs && child.isDirectory(),
|
|
|
- preserveRawXattrs && child.isDirectory());
|
|
|
- writeToFileListing(fileListWriter, childCopyListingStatus,
|
|
|
- sourcePathRoot, options);
|
|
|
- if (child.isDirectory()) {
|
|
|
+ assert numListstatusThreads > 0;
|
|
|
+ LOG.debug("Starting thread pool of " + numListstatusThreads +
|
|
|
+ " listStatus workers.");
|
|
|
+ ProducerConsumer<FileStatus, FileStatus[]> workers =
|
|
|
+ new ProducerConsumer<FileStatus, FileStatus[]>(numListstatusThreads);
|
|
|
+ for (int i = 0; i < numListstatusThreads; i++) {
|
|
|
+ workers.addWorker(
|
|
|
+ new FileStatusProcessor(sourcePathRoot.getFileSystem(getConf())));
|
|
|
+ }
|
|
|
+
|
|
|
+ for (FileStatus status : sourceDirs) {
|
|
|
+ workers.put(new WorkRequest<FileStatus>(status, 0));
|
|
|
+ maybePrintStats();
|
|
|
+ }
|
|
|
+
|
|
|
+ while (workers.hasWork()) {
|
|
|
+ try {
|
|
|
+ WorkReport<FileStatus[]> workResult = workers.take();
|
|
|
+ int retry = workResult.getRetry();
|
|
|
+ for (FileStatus child: workResult.getItem()) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Traversing into source dir: " + child.getPath());
|
|
|
+ LOG.debug("Recording source-path: " + child.getPath() + " for copy.");
|
|
|
+ }
|
|
|
+ if (retry == 0) {
|
|
|
+ CopyListingFileStatus childCopyListingStatus =
|
|
|
+ DistCpUtils.toCopyListingFileStatus(sourceFS, child,
|
|
|
+ preserveAcls && child.isDirectory(),
|
|
|
+ preserveXAttrs && child.isDirectory(),
|
|
|
+ preserveRawXattrs && child.isDirectory());
|
|
|
+ writeToFileListing(fileListWriter, childCopyListingStatus,
|
|
|
+ sourcePathRoot, options);
|
|
|
+ }
|
|
|
+ if (retry < maxRetries) {
|
|
|
+ if (child.isDirectory()) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Traversing into source dir: " + child.getPath());
|
|
|
+ }
|
|
|
+ workers.put(new WorkRequest<FileStatus>(child, retry));
|
|
|
+ maybePrintStats();
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ LOG.error("Giving up on " + child.getPath() +
|
|
|
+ " after " + retry + " retries.");
|
|
|
}
|
|
|
- pathStack.push(child);
|
|
|
}
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ LOG.error("Could not get item from childQueue. Retrying...");
|
|
|
}
|
|
|
}
|
|
|
+ workers.shutdown();
|
|
|
}
|
|
|
|
|
|
private void writeToFileListingRoot(SequenceFile.Writer fileListWriter,
|
|
@@ -351,6 +468,8 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
|
|
|
if (!fileStatus.isDirectory()) {
|
|
|
totalBytesToCopy += fileStatus.getLen();
|
|
|
+ } else {
|
|
|
+ totalDirs++;
|
|
|
}
|
|
|
totalPaths++;
|
|
|
}
|