private Path computeSourceRootPath(FileStatus sourceStatus, DistCpOptions options)
      throws IOException {

    Path target = options.getTargetPath();
    FileSystem targetFS = target.getFileSystem(getConf());

    boolean solitaryFile = options.getSourcePaths().size() == 1 && !sourceStatus.isDir();

    if (solitaryFile) {
      if (targetFS.isFile(target) || !targetFS.exists(target)) {
        return sourceStatus.getPath();
      } else {
        return sourceStatus.getPath().getParent();
      }
    } else {
      boolean specialHandling =
          (options.getSourcePaths().size() == 1 && !targetFS.exists(target))
              || options.shouldSyncFolder()
              || options.shouldOverwrite();

      return specialHandling && sourceStatus.isDir()
          ? sourceStatus.getPath()
          : sourceStatus.getPath().getParent();
    }
  }
  @Override
  protected void validatePaths(DistCpOptions options) throws IOException, InvalidInputException {

    Path targetPath = options.getTargetPath();
    FileSystem targetFS = targetPath.getFileSystem(getConf());
    boolean targetIsFile = targetFS.isFile(targetPath);

    // If target is a file, then source has to be single file
    if (targetIsFile) {
      if (options.getSourcePaths().size() > 1) {
        throw new InvalidInputException("Multiple source being copied to a file: " + targetPath);
      }

      Path srcPath = options.getSourcePaths().get(0);
      FileSystem sourceFS = srcPath.getFileSystem(getConf());
      if (!sourceFS.isFile(srcPath)) {
        throw new InvalidInputException(
            "Cannot copy " + srcPath + ", which is not a file to " + targetPath);
      }
    }

    for (Path path : options.getSourcePaths()) {
      FileSystem fs = path.getFileSystem(getConf());
      if (!fs.exists(path)) {
        throw new InvalidInputException(path + " doesn't exist");
      }
    }

    /* This is requires to allow map tasks to access each of the source
      clusters. This would retrieve the delegation token for each unique
      file system and add them to job's private credential store
    */
    Credentials credentials = getCredentials();
    if (credentials != null) {
      Path[] inputPaths = options.getSourcePaths().toArray(new Path[1]);
      TokenCache.obtainTokensForNamenodes(credentials, inputPaths, getConf());
    }
  }
  /** {@inheritDoc} */
  @Override
  public void doBuildListing(Path pathToListingFile, DistCpOptions options) throws IOException {

    SequenceFile.Writer fileListWriter = null;

    try {
      fileListWriter = getWriter(pathToListingFile);

      for (Path path : options.getSourcePaths()) {
        FileSystem sourceFS = path.getFileSystem(getConf());
        path = makeQualified(path);

        FileStatus rootStatus = sourceFS.getFileStatus(path);
        Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
        boolean localFile = (rootStatus.getClass() != FileStatus.class);

        FileStatus[] sourceFiles = sourceFS.listStatus(path);
        if (sourceFiles != null && sourceFiles.length > 0) {
          for (FileStatus sourceStatus : sourceFiles) {
            if (LOG.isDebugEnabled()) {
              LOG.debug("Recording source-path: " + sourceStatus.getPath() + " for copy.");
            }
            writeToFileListing(fileListWriter, sourceStatus, sourcePathRoot, localFile);

            if (isDirectoryAndNotEmpty(sourceFS, sourceStatus)) {
              if (LOG.isDebugEnabled()) {
                LOG.debug("Traversing non-empty source dir: " + sourceStatus.getPath());
              }
              traverseNonEmptyDirectory(fileListWriter, sourceStatus, sourcePathRoot, localFile);
            }
          }
        } else {
          writeToFileListing(fileListWriter, rootStatus, sourcePathRoot, localFile);
        }
      }
    } finally {
      try {
        if (fileListWriter != null) fileListWriter.close();
      } catch (IOException exception) {
        LOG.error("Could not close output-steam to the file-list: ", exception);
        throw exception;
      }
    }
  }