|
@@ -25,6 +25,8 @@ import org.slf4j.LoggerFactory;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
|
+import org.apache.hadoop.fs.RemoteIterator;
|
|
|
|
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
import org.apache.hadoop.io.SequenceFile;
|
|
@@ -36,6 +38,8 @@ import org.apache.hadoop.tools.util.ProducerConsumer;
|
|
import org.apache.hadoop.tools.util.WorkReport;
|
|
import org.apache.hadoop.tools.util.WorkReport;
|
|
import org.apache.hadoop.tools.util.WorkRequest;
|
|
import org.apache.hadoop.tools.util.WorkRequest;
|
|
import org.apache.hadoop.tools.util.WorkRequestProcessor;
|
|
import org.apache.hadoop.tools.util.WorkRequestProcessor;
|
|
|
|
+import org.apache.hadoop.util.DurationInfo;
|
|
|
|
+import org.apache.hadoop.util.functional.RemoteIterators;
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
|
|
|
@@ -49,6 +53,7 @@ import java.util.HashSet;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Random;
|
|
import java.util.Random;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
|
|
+import java.util.Stack;
|
|
|
|
|
|
import static org.apache.hadoop.tools.DistCpConstants
|
|
import static org.apache.hadoop.tools.DistCpConstants
|
|
.HDFS_RESERVED_RAW_DIRECTORY_NAME;
|
|
.HDFS_RESERVED_RAW_DIRECTORY_NAME;
|
|
@@ -94,11 +99,9 @@ public class SimpleCopyListing extends CopyListing {
|
|
randomizeFileListing = getConf().getBoolean(
|
|
randomizeFileListing = getConf().getBoolean(
|
|
DistCpConstants.CONF_LABEL_SIMPLE_LISTING_RANDOMIZE_FILES,
|
|
DistCpConstants.CONF_LABEL_SIMPLE_LISTING_RANDOMIZE_FILES,
|
|
DEFAULT_RANDOMIZE_FILE_LISTING);
|
|
DEFAULT_RANDOMIZE_FILE_LISTING);
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("numListstatusThreads=" + numListstatusThreads
|
|
|
|
- + ", fileStatusLimit=" + fileStatusLimit
|
|
|
|
- + ", randomizeFileListing=" + randomizeFileListing);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug(
|
|
|
|
+ "numListstatusThreads={}, fileStatusLimit={}, randomizeFileListing={}",
|
|
|
|
+ numListstatusThreads, fileStatusLimit, randomizeFileListing);
|
|
copyFilter = CopyFilter.getCopyFilter(getConf());
|
|
copyFilter = CopyFilter.getCopyFilter(getConf());
|
|
copyFilter.initialize();
|
|
copyFilter.initialize();
|
|
}
|
|
}
|
|
@@ -286,10 +289,8 @@ public class SimpleCopyListing extends CopyListing {
|
|
|
|
|
|
FileStatus sourceStatus = sourceFS.getFileStatus(diff.getTarget());
|
|
FileStatus sourceStatus = sourceFS.getFileStatus(diff.getTarget());
|
|
if (sourceStatus.isDirectory()) {
|
|
if (sourceStatus.isDirectory()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding source dir for traverse: " +
|
|
|
|
- sourceStatus.getPath());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding source dir for traverse: {}",
|
|
|
|
+ sourceStatus.getPath());
|
|
|
|
|
|
HashSet<String> excludeList =
|
|
HashSet<String> excludeList =
|
|
distCpSync.getTraverseExcludeList(diff.getSource(),
|
|
distCpSync.getTraverseExcludeList(diff.getSource(),
|
|
@@ -298,8 +299,9 @@ public class SimpleCopyListing extends CopyListing {
|
|
ArrayList<FileStatus> sourceDirs = new ArrayList<>();
|
|
ArrayList<FileStatus> sourceDirs = new ArrayList<>();
|
|
sourceDirs.add(sourceStatus);
|
|
sourceDirs.add(sourceStatus);
|
|
|
|
|
|
- traverseDirectory(fileListWriter, sourceFS, sourceDirs,
|
|
|
|
- sourceRoot, context, excludeList, fileStatuses);
|
|
|
|
|
|
+ new TraverseDirectory(fileListWriter, sourceFS, sourceDirs,
|
|
|
|
+ sourceRoot, context, excludeList, fileStatuses)
|
|
|
|
+ .traverseDirectory();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -366,9 +368,8 @@ public class SimpleCopyListing extends CopyListing {
|
|
if (explore) {
|
|
if (explore) {
|
|
ArrayList<FileStatus> sourceDirs = new ArrayList<FileStatus>();
|
|
ArrayList<FileStatus> sourceDirs = new ArrayList<FileStatus>();
|
|
for (FileStatus sourceStatus: sourceFiles) {
|
|
for (FileStatus sourceStatus: sourceFiles) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Recording source-path: " + sourceStatus.getPath() + " for copy.");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Recording source-path: {} for copy.",
|
|
|
|
+ sourceStatus.getPath());
|
|
LinkedList<CopyListingFileStatus> sourceCopyListingStatus =
|
|
LinkedList<CopyListingFileStatus> sourceCopyListingStatus =
|
|
DistCpUtils.toCopyListingFileStatus(sourceFS, sourceStatus,
|
|
DistCpUtils.toCopyListingFileStatus(sourceFS, sourceStatus,
|
|
preserveAcls && sourceStatus.isDirectory(),
|
|
preserveAcls && sourceStatus.isDirectory(),
|
|
@@ -384,14 +385,13 @@ public class SimpleCopyListing extends CopyListing {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (sourceStatus.isDirectory()) {
|
|
if (sourceStatus.isDirectory()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding source dir for traverse: " + sourceStatus.getPath());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding source dir for traverse: {}",
|
|
|
|
+ sourceStatus.getPath());
|
|
sourceDirs.add(sourceStatus);
|
|
sourceDirs.add(sourceStatus);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- traverseDirectory(fileListWriter, sourceFS, sourceDirs,
|
|
|
|
- sourcePathRoot, context, null, statusList);
|
|
|
|
|
|
+ new TraverseDirectory(fileListWriter, sourceFS, sourceDirs,
|
|
|
|
+ sourcePathRoot, context, null, statusList).traverseDirectory();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (randomizeFileListing) {
|
|
if (randomizeFileListing) {
|
|
@@ -429,16 +429,12 @@ public class SimpleCopyListing extends CopyListing {
|
|
*/
|
|
*/
|
|
Collections.shuffle(fileStatusInfoList, rnd);
|
|
Collections.shuffle(fileStatusInfoList, rnd);
|
|
for (FileStatusInfo fileStatusInfo : fileStatusInfoList) {
|
|
for (FileStatusInfo fileStatusInfo : fileStatusInfoList) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding " + fileStatusInfo.fileStatus.getPath());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding {}", fileStatusInfo.fileStatus.getPath());
|
|
writeToFileListing(fileListWriter, fileStatusInfo.fileStatus,
|
|
writeToFileListing(fileListWriter, fileStatusInfo.fileStatus,
|
|
fileStatusInfo.sourceRootPath);
|
|
fileStatusInfo.sourceRootPath);
|
|
}
|
|
}
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Number of paths written to fileListing="
|
|
|
|
- + fileStatusInfoList.size());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Number of paths written to fileListing={}",
|
|
|
|
+ fileStatusInfoList.size());
|
|
fileStatusInfoList.clear();
|
|
fileStatusInfoList.clear();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -590,8 +586,8 @@ public class SimpleCopyListing extends CopyListing {
|
|
result = new WorkReport<FileStatus[]>(getFileStatus(parent.getPath()),
|
|
result = new WorkReport<FileStatus[]>(getFileStatus(parent.getPath()),
|
|
retry, true);
|
|
retry, true);
|
|
} catch (FileNotFoundException fnf) {
|
|
} catch (FileNotFoundException fnf) {
|
|
- LOG.error("FileNotFoundException exception in listStatus: " +
|
|
|
|
- fnf.getMessage());
|
|
|
|
|
|
+ LOG.error("FileNotFoundException exception in listStatus: {}",
|
|
|
|
+ fnf.getMessage());
|
|
result = new WorkReport<FileStatus[]>(new FileStatus[0], retry, true,
|
|
result = new WorkReport<FileStatus[]>(new FileStatus[0], retry, true,
|
|
fnf);
|
|
fnf);
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
@@ -605,8 +601,7 @@ public class SimpleCopyListing extends CopyListing {
|
|
}
|
|
}
|
|
|
|
|
|
private void printStats() {
|
|
private void printStats() {
|
|
- LOG.info("Paths (files+dirs) cnt = " + totalPaths +
|
|
|
|
- "; dirCnt = " + totalDirs);
|
|
|
|
|
|
+ LOG.info("Paths (files+dirs) cnt = {}; dirCnt = ", totalPaths, totalDirs);
|
|
}
|
|
}
|
|
|
|
|
|
private void maybePrintStats() {
|
|
private void maybePrintStats() {
|
|
@@ -615,79 +610,6 @@ public class SimpleCopyListing extends CopyListing {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private void traverseDirectory(SequenceFile.Writer fileListWriter,
|
|
|
|
- FileSystem sourceFS,
|
|
|
|
- ArrayList<FileStatus> sourceDirs,
|
|
|
|
- Path sourcePathRoot,
|
|
|
|
- DistCpContext context,
|
|
|
|
- HashSet<String> excludeList,
|
|
|
|
- List<FileStatusInfo> fileStatuses)
|
|
|
|
- throws IOException {
|
|
|
|
- final boolean preserveAcls = context.shouldPreserve(FileAttribute.ACL);
|
|
|
|
- final boolean preserveXAttrs = context.shouldPreserve(FileAttribute.XATTR);
|
|
|
|
- final boolean preserveRawXattrs = context.shouldPreserveRawXattrs();
|
|
|
|
-
|
|
|
|
- assert numListstatusThreads > 0;
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- 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()),
|
|
|
|
- excludeList));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- for (FileStatus status : sourceDirs) {
|
|
|
|
- workers.put(new WorkRequest<FileStatus>(status, 0));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- while (workers.hasWork()) {
|
|
|
|
- try {
|
|
|
|
- WorkReport<FileStatus[]> workResult = workers.take();
|
|
|
|
- int retry = workResult.getRetry();
|
|
|
|
- for (FileStatus child: workResult.getItem()) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Recording source-path: " + child.getPath() + " for copy.");
|
|
|
|
- }
|
|
|
|
- if (workResult.getSuccess()) {
|
|
|
|
- LinkedList<CopyListingFileStatus> childCopyListingStatus =
|
|
|
|
- DistCpUtils.toCopyListingFileStatus(sourceFS, child,
|
|
|
|
- preserveAcls && child.isDirectory(),
|
|
|
|
- preserveXAttrs && child.isDirectory(),
|
|
|
|
- preserveRawXattrs && child.isDirectory(),
|
|
|
|
- context.getBlocksPerChunk());
|
|
|
|
-
|
|
|
|
- for (CopyListingFileStatus fs : childCopyListingStatus) {
|
|
|
|
- if (randomizeFileListing) {
|
|
|
|
- addToFileListing(fileStatuses,
|
|
|
|
- new FileStatusInfo(fs, sourcePathRoot), fileListWriter);
|
|
|
|
- } else {
|
|
|
|
- writeToFileListing(fileListWriter, fs, sourcePathRoot);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (retry < maxRetries) {
|
|
|
|
- if (child.isDirectory()) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Traversing into source dir: " + child.getPath());
|
|
|
|
- }
|
|
|
|
- workers.put(new WorkRequest<FileStatus>(child, retry));
|
|
|
|
- }
|
|
|
|
- } else {
|
|
|
|
- LOG.error("Giving up on " + child.getPath() +
|
|
|
|
- " after " + retry + " retries.");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
- LOG.error("Could not get item from childQueue. Retrying...");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- workers.shutdown();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
private void writeToFileListingRoot(SequenceFile.Writer fileListWriter,
|
|
private void writeToFileListingRoot(SequenceFile.Writer fileListWriter,
|
|
LinkedList<CopyListingFileStatus> fileStatus, Path sourcePathRoot,
|
|
LinkedList<CopyListingFileStatus> fileStatus, Path sourcePathRoot,
|
|
DistCpContext context) throws IOException {
|
|
DistCpContext context) throws IOException {
|
|
@@ -697,9 +619,7 @@ public class SimpleCopyListing extends CopyListing {
|
|
if (fs.getPath().equals(sourcePathRoot) &&
|
|
if (fs.getPath().equals(sourcePathRoot) &&
|
|
fs.isDirectory() && syncOrOverwrite) {
|
|
fs.isDirectory() && syncOrOverwrite) {
|
|
// Skip the root-paths when syncOrOverwrite
|
|
// Skip the root-paths when syncOrOverwrite
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Skip " + fs.getPath());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Skip {}", fs.getPath());
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
writeToFileListing(fileListWriter, fs, sourcePathRoot);
|
|
writeToFileListing(fileListWriter, fs, sourcePathRoot);
|
|
@@ -709,10 +629,9 @@ public class SimpleCopyListing extends CopyListing {
|
|
private void writeToFileListing(SequenceFile.Writer fileListWriter,
|
|
private void writeToFileListing(SequenceFile.Writer fileListWriter,
|
|
CopyListingFileStatus fileStatus,
|
|
CopyListingFileStatus fileStatus,
|
|
Path sourcePathRoot) throws IOException {
|
|
Path sourcePathRoot) throws IOException {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("REL PATH: " + DistCpUtils.getRelativePath(sourcePathRoot,
|
|
|
|
- fileStatus.getPath()) + ", FULL PATH: " + fileStatus.getPath());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("REL PATH: {}, FULL PATH: {}",
|
|
|
|
+ DistCpUtils.getRelativePath(sourcePathRoot, fileStatus.getPath()),
|
|
|
|
+ fileStatus.getPath());
|
|
|
|
|
|
if (!shouldCopy(fileStatus.getPath())) {
|
|
if (!shouldCopy(fileStatus.getPath())) {
|
|
return;
|
|
return;
|
|
@@ -730,4 +649,159 @@ public class SimpleCopyListing extends CopyListing {
|
|
totalPaths++;
|
|
totalPaths++;
|
|
maybePrintStats();
|
|
maybePrintStats();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * A utility class to traverse a directory.
|
|
|
|
+ */
|
|
|
|
+ private final class TraverseDirectory {
|
|
|
|
+
|
|
|
|
+ private SequenceFile.Writer fileListWriter;
|
|
|
|
+ private FileSystem sourceFS;
|
|
|
|
+ private ArrayList<FileStatus> sourceDirs;
|
|
|
|
+ private Path sourcePathRoot;
|
|
|
|
+ private DistCpContext context;
|
|
|
|
+ private HashSet<String> excludeList;
|
|
|
|
+ private List<FileStatusInfo> fileStatuses;
|
|
|
|
+ private final boolean preserveAcls;
|
|
|
|
+ private final boolean preserveXAttrs;
|
|
|
|
+ private final boolean preserveRawXattrs;
|
|
|
|
+
|
|
|
|
+ private TraverseDirectory(SequenceFile.Writer fileListWriter,
|
|
|
|
+ FileSystem sourceFS, ArrayList<FileStatus> sourceDirs,
|
|
|
|
+ Path sourcePathRoot, DistCpContext context, HashSet<String> excludeList,
|
|
|
|
+ List<FileStatusInfo> fileStatuses) {
|
|
|
|
+ this.fileListWriter = fileListWriter;
|
|
|
|
+ this.sourceFS = sourceFS;
|
|
|
|
+ this.sourceDirs = sourceDirs;
|
|
|
|
+ this.sourcePathRoot = sourcePathRoot;
|
|
|
|
+ this.context = context;
|
|
|
|
+ this.excludeList = excludeList;
|
|
|
|
+ this.fileStatuses = fileStatuses;
|
|
|
|
+ this.preserveAcls = context.shouldPreserve(FileAttribute.ACL);
|
|
|
|
+ this.preserveXAttrs = context.shouldPreserve(FileAttribute.XATTR);
|
|
|
|
+ this.preserveRawXattrs = context.shouldPreserveRawXattrs();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void traverseDirectory() throws IOException {
|
|
|
|
+ if (context.shouldUseIterator()) {
|
|
|
|
+ try (DurationInfo ignored = new DurationInfo(LOG,
|
|
|
|
+ "Building listing using iterator mode for %s", sourcePathRoot)) {
|
|
|
|
+ traverseDirectoryLegacy();
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ try (DurationInfo ignored = new DurationInfo(LOG,
|
|
|
|
+ "Building listing using multi threaded approach for %s",
|
|
|
|
+ sourcePathRoot)) {
|
|
|
|
+ traverseDirectoryMultiThreaded();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void traverseDirectoryMultiThreaded() throws IOException {
|
|
|
|
+ assert numListstatusThreads > 0;
|
|
|
|
+
|
|
|
|
+ LOG.debug("Starting thread pool of {} listStatus workers.",
|
|
|
|
+ numListstatusThreads);
|
|
|
|
+
|
|
|
|
+ ProducerConsumer<FileStatus, FileStatus[]> workers =
|
|
|
|
+ new ProducerConsumer<FileStatus, FileStatus[]>(numListstatusThreads);
|
|
|
|
+ try {
|
|
|
|
+ for (int i = 0; i < numListstatusThreads; i++) {
|
|
|
|
+ workers.addWorker(
|
|
|
|
+ new FileStatusProcessor(sourcePathRoot.getFileSystem(getConf()),
|
|
|
|
+ excludeList));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ for (FileStatus status : sourceDirs) {
|
|
|
|
+ workers.put(new WorkRequest<FileStatus>(status, 0));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ while (workers.hasWork()) {
|
|
|
|
+ try {
|
|
|
|
+ WorkReport<FileStatus[]> workResult = workers.take();
|
|
|
|
+ int retry = workResult.getRetry();
|
|
|
|
+ for (FileStatus child : workResult.getItem()) {
|
|
|
|
+ LOG.debug("Recording source-path: {} for copy.", child.getPath());
|
|
|
|
+ boolean isChildDirectory = child.isDirectory();
|
|
|
|
+ if (workResult.getSuccess()) {
|
|
|
|
+ LinkedList<CopyListingFileStatus> childCopyListingStatus =
|
|
|
|
+ DistCpUtils.toCopyListingFileStatus(sourceFS, child,
|
|
|
|
+ preserveAcls && isChildDirectory,
|
|
|
|
+ preserveXAttrs && isChildDirectory,
|
|
|
|
+ preserveRawXattrs && isChildDirectory,
|
|
|
|
+ context.getBlocksPerChunk());
|
|
|
|
+
|
|
|
|
+ for (CopyListingFileStatus fs : childCopyListingStatus) {
|
|
|
|
+ if (randomizeFileListing) {
|
|
|
|
+ addToFileListing(fileStatuses,
|
|
|
|
+ new FileStatusInfo(fs, sourcePathRoot), fileListWriter);
|
|
|
|
+ } else {
|
|
|
|
+ writeToFileListing(fileListWriter, fs, sourcePathRoot);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (retry < maxRetries) {
|
|
|
|
+ if (isChildDirectory) {
|
|
|
|
+ LOG.debug("Traversing into source dir: {}", child.getPath());
|
|
|
|
+ workers.put(new WorkRequest<FileStatus>(child, retry));
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ LOG.error("Giving up on {} after {} retries.", child.getPath(),
|
|
|
|
+ retry);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
|
+ LOG.error("Could not get item from childQueue. Retrying...");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ workers.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void traverseDirectoryLegacy() throws IOException {
|
|
|
|
+ Stack<FileStatus> pathStack = new Stack<FileStatus>();
|
|
|
|
+ for (FileStatus fs : sourceDirs) {
|
|
|
|
+ if (excludeList == null || !excludeList
|
|
|
|
+ .contains(fs.getPath().toUri().getPath())) {
|
|
|
|
+ pathStack.add(fs);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ while (!pathStack.isEmpty()) {
|
|
|
|
+ prepareListing(pathStack.pop().getPath());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void prepareListing(Path path) throws IOException {
|
|
|
|
+ LOG.debug("Recording source-path: {} for copy.", path);
|
|
|
|
+ RemoteIterator<FileStatus> listStatus = RemoteIterators
|
|
|
|
+ .filteringRemoteIterator(sourceFS.listStatusIterator(path),
|
|
|
|
+ i -> excludeList == null || !excludeList
|
|
|
|
+ .contains(i.getPath().toUri().getPath()));
|
|
|
|
+ while (listStatus.hasNext()) {
|
|
|
|
+ FileStatus child = listStatus.next();
|
|
|
|
+ LinkedList<CopyListingFileStatus> childCopyListingStatus = DistCpUtils
|
|
|
|
+ .toCopyListingFileStatus(sourceFS, child,
|
|
|
|
+ preserveAcls && child.isDirectory(),
|
|
|
|
+ preserveXAttrs && child.isDirectory(),
|
|
|
|
+ preserveRawXattrs && child.isDirectory(),
|
|
|
|
+ context.getBlocksPerChunk());
|
|
|
|
+ for (CopyListingFileStatus fs : childCopyListingStatus) {
|
|
|
|
+ if (randomizeFileListing) {
|
|
|
|
+ addToFileListing(fileStatuses,
|
|
|
|
+ new FileStatusInfo(fs, sourcePathRoot), fileListWriter);
|
|
|
|
+ } else {
|
|
|
|
+ writeToFileListing(fileListWriter, fs, sourcePathRoot);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (child.isDirectory()) {
|
|
|
|
+ LOG.debug("Traversing into source dir: {}", child.getPath());
|
|
|
|
+ prepareListing(child.getPath());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ IOStatisticsLogging
|
|
|
|
+ .logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}",
|
|
|
|
+ listStatus);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|