/**
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.hadoop.tools;

import java.io.BufferedReader;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStreamReader;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import java.util.Stack;
import java.util.StringTokenizer;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.tools.FastCopy;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.mapred.FileOutputFormat;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.InvalidInputException;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Mapper;
import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reducer;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.RunningJob;
import org.apache.hadoop.mapred.SequenceFileRecordReader;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;

/**
 * A Map-reduce program to recursively copy directories between
 * different file-systems.
 */
public class DistCp implements Tool {
  public static final Log LOG = LogFactory.getLog(DistCp.class);

  private static final String NAME = "distcp";

  private static final String usage = NAME
    + " [OPTIONS] <srcurl>* <desturl>" +
    "\n\nOPTIONS:" +
    "\n-p[rbugp]              Preserve status" +
    "\n                       r: replication number" +
    "\n                       b: block size" +
    "\n                       u: user" + 
    "\n                       g: group" +
    "\n                       p: permission" +
    "\n                       t: modification and access times" +
    "\n                       -p alone is equivalent to -prbugpt" +
    "\n-i                     Ignore failures" +
    "\n-basedir <basedir>     Use <basedir> as the base directory when copying files from <srcurl>" +
    "\n-log <logdir>          Write logs to <logdir>" +
    "\n-m <num_maps>          Maximum number of simultaneous copies" +
    "\n-r <num_reducers>      Maximum number of reducers" +
    "\n-overwrite             Overwrite destination" +
    "\n-update                Overwrite if src size different from dst size" +
    "\n-skipcrccheck          Do not use CRC check to determine if src is " +
    "\n                       different from dest. Relevant only if -update" +
    "\n                       is specified" +
    "\n-copybychunk           Chop files in chunk and copy" +
    "\n-f <urilist_uri>       Use list at <urilist_uri> as src list" +
    "\n-filelimit <n>         Limit the total number of files to be <= n" +
    "\n-sizelimit <n>         Limit the total size to be <= n bytes" +
    "\n-delete                Delete the files existing in the dst but not in src" +
    "\n-mapredSslConf <f>     Filename of SSL configuration for mapper task" +
    "\n-usefastcopy           Use FastCopy (applicable to DFS only)" +
    
    "\n\nNOTE 1: if -overwrite or -update are set, each source URI is " +
    "\n      interpreted as an isomorphic update to an existing directory." +
    "\nFor example:" +
    "\nhadoop " + NAME + " -p -update \"hdfs://A:8020/user/foo/bar\" " +
    "\"hdfs://B:8020/user/foo/baz\"\n" +
    "\n     would update all descendants of 'baz' also in 'bar'; it would " +
    "\n     *not* update /user/foo/baz/bar" + 

    "\n\nNOTE 2: The parameter <n> in -filelimit and -sizelimit can be " +
    "\n     specified with symbolic representation.  For examples," +
    "\n       1230k = 1230 * 1024 = 1259520" +
    "\n       891g = 891 * 1024^3 = 956703965184" +
    
    "\n";
  
  private static final long BYTES_PER_MAP =  256 * 1024 * 1024;
  private static final int MAX_MAPS_PER_NODE = 20;
  private static final int MAX_MAPS_DEFAULT = 4000;
  private static final int SYNC_FILE_MAX = 10;

  static enum Counter {
    COPY, SKIP, FAIL, BYTESCOPIED, BYTESEXPECTED, BLOCKSCOPIED
  }

  public static enum Options {
    DELETE("-delete", NAME + ".delete"),
    FILE_LIMIT("-filelimit", NAME + ".limit.file"),
    SIZE_LIMIT("-sizelimit", NAME + ".limit.size"),
    IGNORE_READ_FAILURES("-i", NAME + ".ignore.read.failures"),
    PRESERVE_STATUS("-p", NAME + ".preserve.status"),
    OVERWRITE("-overwrite", NAME + ".overwrite.always"),
    UPDATE("-update", NAME + ".overwrite.ifnewer"),
    SKIPCRC("-skipcrccheck", NAME + ".skip.crc.check"),
    COPYBYCHUNK("-copybychunk", NAME + ".copy.by.chunk"),
    USEFASTCOPY("-usefastcopy", NAME + ".use.fastcopy");

    final String cmd, propertyname;

    private Options(String cmd, String propertyname) {
      this.cmd = cmd;
      this.propertyname = propertyname;
    }
    
    private long parseLong(String[] args, int offset) {
      if (offset ==  args.length) {
        throw new IllegalArgumentException("<n> not specified in " + cmd);
      }
      long n = StringUtils.TraditionalBinaryPrefix.string2long(args[offset]);
      if (n <= 0) {
        throw new IllegalArgumentException("n = " + n + " <= 0 in " + cmd);
      }
      return n;
    }
  }
  static enum FileAttribute {
    BLOCK_SIZE, REPLICATION, USER, GROUP, PERMISSION, TIMES;

    final char symbol;

    private FileAttribute() {symbol = toString().toLowerCase().charAt(0);}
    
    static EnumSet<FileAttribute> parse(String s) {
      if (s == null || s.length() == 0) {
        return EnumSet.allOf(FileAttribute.class);
      }

      EnumSet<FileAttribute> set = EnumSet.noneOf(FileAttribute.class);
      FileAttribute[] attributes = values();
      for(char c : s.toCharArray()) {
        int i = 0;
        for(; i < attributes.length && c != attributes[i].symbol; i++);
        if (i < attributes.length) {
          if (!set.contains(attributes[i])) {
            set.add(attributes[i]);
          } else {
            throw new IllegalArgumentException("There are more than one '"
                + attributes[i].symbol + "' in " + s); 
          }
        } else {
          throw new IllegalArgumentException("'" + c + "' in " + s
              + " is undefined.");
        }
      }
      return set;
    }
  }

  static final String TMP_DIR_LABEL = NAME + ".tmp.dir";
  static final String DST_DIR_LABEL = NAME + ".dest.path";
  static final String JOB_DIR_LABEL = NAME + ".job.dir";
  static final String MAX_MAPS_LABEL = NAME + ".max.map.tasks";
  static final String MAX_REDUCE_LABEL = NAME + ".max.reduce.tasks";
  static final String SRC_LIST_LABEL = NAME + ".src.list";
  static final String SRC_COUNT_LABEL = NAME + ".src.count";
  static final String TOTAL_SIZE_LABEL = NAME + ".total.size";
  static final String TOTAL_BLOCKS_LABEL = NAME + ".total.blocks";
  static final String DST_DIR_LIST_LABEL = NAME + ".dst.dir.list";
  static final String DST_CHUNK_FILE_LIST_LABEL = NAME + ".dst.chunk.file.list";
  static final String SPLIT_LIST_LABEL = NAME + ".file.split.list";
  public static final String BYTES_PER_MAP_LABEL = NAME + ".bytes.per.map";
  static final String PRESERVE_STATUS_LABEL
      = Options.PRESERVE_STATUS.propertyname + ".value";

  private JobConf conf;

  public void setConf(Configuration conf) {
    if (conf instanceof JobConf) {
      this.conf = (JobConf) conf;
    } else {
      this.conf = new JobConf(conf);
    }
  }

  public Configuration getConf() {
    return conf;
  }

  public DistCp(Configuration conf) {
    setConf(conf);
  }

  /**
   * An input/output pair of filenames.
   * offset and length field are used for chopping files into chunk files
   * and copying file chunks instead of one big file
   * offset tells where is the starting point of the chuck file in the 
   * original file 
   * length tells the length of the chuck file
   * If the FilePair are directories or we don't want to copy file by chunks,
   * both offset and length will be set to 0
   * Also we use the index to tell the order of the chunks
   *
   */
  static class FileChunkPair implements Writable {
    FileStatus input = new FileStatus();
    String output;
    long offset;
    long length;
    int chunkIndex;
    FileChunkPair() { }
    FileChunkPair(FileStatus input, String output, long offset, 
        long length, int chunkIndex) {
      this.input = input;
      this.output = output;
      this.offset = offset;
      this.length = length;
      this.chunkIndex = chunkIndex;
    }
    public void readFields(DataInput in) throws IOException {
      input.readFields(in);
      output = Text.readString(in);
      offset = in.readLong();
      length = in.readLong();
      chunkIndex = in.readInt();
    }
    public void write(DataOutput out) throws IOException {
      input.write(out);
      Text.writeString(out, output);
      out.writeLong(offset);
      out.writeLong(length);
      out.writeInt(chunkIndex);
    }
    public String toString() {
      return input + " : " + output + " : " + offset + " : " + length  
      + " : " + chunkIndex ;
    }
  }
  
  /**
   * An input/output pair of filenames.
   */  
  static class FilePairComparable implements WritableComparable {
    FileStatus input = new FileStatus();
    String output;
    FilePairComparable() { }
    FilePairComparable(FileStatus input, String output) {
      this.input = input;
      this.output = output;
    }
    public int compareTo(Object otherObj) { 
      FilePairComparable other = (FilePairComparable) otherObj;
      return this.input.compareTo(other.input);
    }
    public void readFields(DataInput in) throws IOException {
      input.readFields(in);
      output = Text.readString(in);
    }
    public void write(DataOutput out) throws IOException {
      input.write(out);
      Text.writeString(out, output);
    }
    public String toString() {
      return input + " : " + output;
    }
  }

  /**
   * Computes the total number of blocks in a file.
   * 
   * @param len
   *          the length of the file.
   * @param blockSize
   *          the block size for the file.
   * @return the total number of blocks.
   */
  private static long getBlocks(long len, long blockSize) {
    return (len == 0 || blockSize == 0) ? 0 : (len - 1) / blockSize + 1;
  }

  /**
   * InputFormat of a distcp job responsible for generating splits of the src
   * file list when FastCopy is being used for copying.
   */
  static class FastCopyInputFormat extends CopyInputFormat {

    @Override
    protected long getIncrement(LongWritable key, FilePairComparable value) {
      return getBlocks(value.input.getLen(), value.input.getBlockSize());
    }

    @Override
    protected long getTargetSize(JobConf job, int numSplits) {
      long totalBlocks = job.getLong(TOTAL_BLOCKS_LABEL, -1);
      return totalBlocks / numSplits;
    }
  }
  
  /**
   * InputFormat of a distcp job responsible for generating splits of the src
   * file list.
   */
  static class CopyInputFormat implements InputFormat<Text, Text> {

    protected long getIncrement(LongWritable key, FilePairComparable value) {
      return key.get();
    }

    protected long getTargetSize(JobConf job, int numSplits) {
      long cbsize = job.getLong(TOTAL_SIZE_LABEL, -1);
      return cbsize / numSplits;
    }

    /**
     * Produce splits such that each is no greater than the quotient of the
     * total size and the number of splits requested.
     * @param job The handle to the JobConf object
     * @param numSplits Number of splits requested
     */
    public InputSplit[] getSplits(JobConf job, int numSplits)
        throws IOException {
      int cnfiles = job.getInt(SRC_COUNT_LABEL, -1);
      long cbsize = job.getLong(TOTAL_SIZE_LABEL, -1);
      long blocks = job.getLong(TOTAL_BLOCKS_LABEL, -1);
      String srcfilelist = job.get(SRC_LIST_LABEL, "");
      if (cnfiles < 0 || cbsize < 0 || blocks < 0 || "".equals(srcfilelist)) {
        throw new RuntimeException("Invalid metadata: #files(" + cnfiles +
                                   ") total_size(" + cbsize + ") listuri(" +
                                   srcfilelist + ")");
      }
      Path src = new Path(srcfilelist);
      FileSystem fs = src.getFileSystem(job);
      FileStatus srcst = fs.getFileStatus(src);

      ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits);
      LongWritable key = new LongWritable();
      FilePairComparable value = new FilePairComparable();
      final long targetsize = getTargetSize(job, numSplits);
      long pos = 0L;
      long last = 0L;
      long acc = 0L;
      long cbrem = srcst.getLen();
      SequenceFile.Reader sl = null;
      try {
        sl = new SequenceFile.Reader(fs, src, job);
        for (; sl.next(key, value); last = sl.getPosition()) {
          // if adding this split would put this split past the target size,
          // cut the last split and put this next file in the next split.
          long increment = getIncrement(key, value);
          if (acc + increment > targetsize && acc != 0) {
            long splitsize = last - pos;
            splits.add(new FileSplit(src, pos, splitsize, (String[])null));
            cbrem -= splitsize;
            pos = last;
            acc = 0L;
          }
          acc += increment;
        }
      }
      finally {
        checkAndClose(sl);
      }
      if (cbrem != 0) {
        splits.add(new FileSplit(src, pos, cbrem, (String[])null));
      }

      return splits.toArray(new FileSplit[splits.size()]);
    }

    /**
     * Returns a reader for this split of the src file list.
     */
    public RecordReader<Text, Text> getRecordReader(InputSplit split,
        JobConf job, Reporter reporter) throws IOException {
      return new SequenceFileRecordReader<Text, Text>(job, (FileSplit)split);
    }
  }
  
  /**
   * InputFormat of a distcp job responsible for generating splits from the 
   * split list we create during the setup phase.
   * 
   */
  static class CopyByChunkInputFormat extends CopyInputFormat 
  implements InputFormat<Text, Text> {

    /**
     * Produce splits such that each is no greater than the quotient of the
     * total size and the number of splits requested.
     * @param job The handle to the JobConf object
     * @param numSplits Number of splits requested
     */
    public InputSplit[] getSplits(JobConf job, int numSplits)
        throws IOException {
      int cnfiles = job.getInt(SRC_COUNT_LABEL, -1);
      long cbsize = job.getLong(TOTAL_SIZE_LABEL, -1);
      String srcFileList = job.get(SRC_LIST_LABEL, "");
      Path srcFileListPath = new Path(srcFileList);
      if (cnfiles < 0 || cbsize < 0 || "".equals(srcFileList)) {
        throw new RuntimeException("Invalid metadata: #files(" + cnfiles +
            ") total_size(" + cbsize + ") src_chunk_file_list_uri(" +
            srcFileList + ")");
      }
      ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits);
      SequenceFile.Reader sl = null;
      String splitList = job.get(SPLIT_LIST_LABEL, "");
      if("".equals(splitList)) {
        throw new RuntimeException("Invalid metadata: split_list_uri(" +
            srcFileList + ")");
      }
      //split file list which contains start pos and split length pairs
      //they are used to split srcChunkFileList
      Path splitListPath = new Path(splitList);        
      FileSystem splitListFs = splitListPath.getFileSystem(job);
      try{
        sl = new SequenceFile.Reader(splitListFs, splitListPath, job);
        LongWritable startpos = new LongWritable();
        LongWritable length = new LongWritable();
        while (sl.next(startpos, length)) {
          splits.add(new FileSplit(srcFileListPath, startpos.get(), 
              length.get(), (String[])null));
        }
      }
      finally{
        checkAndClose(sl);
      }
      return splits.toArray(new FileSplit[splits.size()]);
    }
  }
  /**
   * FSCopyFilesTask: The mapper for copying files between FileSystems.
   */
  static class CopyFilesTask
      implements Mapper<LongWritable, FilePairComparable, FilePairComparable, Text>,
                 Reducer<FilePairComparable, Text, FilePairComparable, Text> {
    // config
    private int sizeBuf = 128 * 1024;
    private FileSystem destFileSys = null;
    private boolean ignoreReadFailures;
    private boolean preserve_status;
    private EnumSet<FileAttribute> preserved;
    private boolean overwrite;
    private boolean update;
    private Path destPath = null;
    private Path tmpRoot = null;
    private Path attemptTmpRoot = null;
    private byte[] buffer = null;
    private JobConf job;
    private boolean skipCRCCheck = false;
    private boolean useFastCopy = false;
    
    private FastCopy fc = null;

    // stats
    private int failcount = 0;
    private int skipcount = 0;
    private int copycount = 0;

    private String getCountString() {
      return "Copied: " + copycount + " Skipped: " + skipcount
      + " Failed: " + failcount;
    }
    private void updateStatus(Reporter reporter) {
      reporter.setStatus(getCountString());
    }

    /**
     * Return true if dst should be replaced by src and the update flag is set.
     * Right now, this merely checks that the src and dst len are not equal. 
     * This should be improved on once modification times, CRCs, etc. can
     * be meaningful in this context.
     * @throws IOException 
     */
    private boolean needsUpdate(FileStatus srcstatus,
        FileSystem dstfs, Path dstpath) throws IOException {
      return update && !sameFile(srcstatus.getPath().getFileSystem(job),
          srcstatus, dstfs, dstpath, skipCRCCheck);
    }

    private FSDataOutputStream create(Path f, Reporter reporter,
        FileStatus srcstat) throws IOException {
      if (destFileSys.exists(f)) {
        destFileSys.delete(f, false);
      }
      if (!preserve_status) {
        return destFileSys.create(f, true, sizeBuf, reporter);
      }

      FsPermission permission = preserved.contains(FileAttribute.PERMISSION)?
          srcstat.getPermission(): null;
      short replication = preserved.contains(FileAttribute.REPLICATION)?
          srcstat.getReplication(): destFileSys.getDefaultReplication();
      long blockSize = preserved.contains(FileAttribute.BLOCK_SIZE)?
          srcstat.getBlockSize(): destFileSys.getDefaultBlockSize();
      return destFileSys.create(f, permission, true, sizeBuf, replication,
          blockSize, reporter);
    }

    /**
     * Copy a file to a destination without breaking file into chunks
     * @param filePair the pair of source and dest
     * @param outc map output collector
     * @param reporter
     */
    private void copy(FilePairComparable filePair,
        OutputCollector<FilePairComparable, Text> outc, Reporter reporter)
        throws IOException {
      FileStatus srcstat = filePair.input;
      Path relativedst = new Path(filePair.output);
      Path absdst = new Path(destPath, relativedst);
      Path tmpFile = new Path(attemptTmpRoot, relativedst);
      Path finalTmpFile = new Path(tmpRoot, relativedst);
      int totfiles = job.getInt(SRC_COUNT_LABEL, -1);
      assert totfiles >= 0 : "Invalid file count " + totfiles;

      // if a directory, ensure created even if empty
      if (srcstat.isDir()) {
        if (destFileSys.exists(absdst)) {
          if (!destFileSys.getFileStatus(absdst).isDir()) {
            throw new IOException("Failed to mkdirs: " + absdst+" is a file.");
          }
        }
        else if (!destFileSys.mkdirs(absdst)) {
          throw new IOException("Failed to mkdirs " + absdst);
        }
        // TODO: when modification times can be set, directories should be
        // emitted to reducers so they might be preserved. Also, mkdirs does
        // not currently return an error when the directory already exists;
        // if this changes, all directory work might as well be done in reduce
        return;
      }

      if ((destFileSys.exists(absdst) && !overwrite
          && !needsUpdate(srcstat, destFileSys, absdst)) ||
            destFileSys.exists(finalTmpFile)) {
        outc.collect(filePair, new Text("SKIP: " + srcstat.getPath()));
        ++skipcount;
        reporter.incrCounter(Counter.SKIP, 1);
        updateStatus(reporter);
        return;
      }
      
      FileSystem srcFileSys = srcstat.getPath().getFileSystem(job);
      reporter.incrCounter(Counter.BYTESEXPECTED, srcstat.getLen());
      if (useFastCopy) {
        try {
          if (fc == null) {
            throw new IOException("FastCopy object has not been instantiated.");
          }
          LOG.info("Use FastCopy to copy File from " + srcstat.getPath() +" to " + tmpFile);
          fc.copy(srcstat.getPath().toString(), tmpFile.toString(), 
              DFSUtil.convertToDFS(srcFileSys),
              DFSUtil.convertToDFS(destFileSys), reporter);
        } catch (Exception e) {
          throw new IOException("FastCopy throws exception", e);
        }
        reporter.setStatus("Copied " + srcstat.getPath().toString());
      } else {
        long cbcopied = 0L;
        FSDataInputStream in = null;
        FSDataOutputStream out = null;
        try {
          // open src file
          in = srcstat.getPath().getFileSystem(job).open(srcstat.getPath());
          // open tmp file
          out = create(tmpFile, reporter, srcstat);
          // copy file
          for(int cbread; (cbread = in.read(buffer)) >= 0; ) {
            out.write(buffer, 0, cbread);
            cbcopied += cbread;
            reporter.setStatus(
                String.format("%.2f ", cbcopied*100.0/srcstat.getLen())
                + absdst + " [ " +
                StringUtils.humanReadableInt(cbcopied) + " / " +
                StringUtils.humanReadableInt(srcstat.getLen()) + " ]");
          }
        } finally {
          checkAndClose(in);
          checkAndClose(out);
        }
        
        if (cbcopied != srcstat.getLen()) {
          throw new IOException("File size not matched: copied "
              + bytesString(cbcopied) + " to tmpFile (=" + tmpFile
              + ") but expected " + bytesString(srcstat.getLen()) 
              + " from " + srcstat.getPath());        
        }
        long copiedFile = destFileSys.getFileStatus(tmpFile).getLen();
        if (copiedFile != srcstat.getLen()) {
          throw new IOException("File size not matched: original "
              + bytesString(srcstat.getLen()) + " copied "
              + bytesString(copiedFile)
              + " from " + srcstat.getPath());
        }
      }
      
      if (totfiles == 1) {
        // Copying a single file; use dst path provided by user as destination
        // rather than destination directory, if a file
        Path dstparent = finalTmpFile.getParent();
        if (!(destFileSys.exists(dstparent) &&
              destFileSys.getFileStatus(dstparent).isDir())) {
          finalTmpFile = dstparent;
        }
      }
      if (destFileSys.exists(finalTmpFile) &&
          destFileSys.getFileStatus(finalTmpFile).isDir()) {
        throw new IOException(finalTmpFile + " is a directory");
      }
      if (!destFileSys.mkdirs(finalTmpFile.getParent())) {
        throw new IOException("Failed to create parent dir: " + finalTmpFile.getParent());
      }
      outc.collect(filePair, new Text("COPIED"));
      destFileSys.rename(tmpFile, finalTmpFile);


      // report at least once for each file
      ++copycount;
      reporter.incrCounter(Counter.BYTESCOPIED, srcstat.getLen());
      reporter.incrCounter(Counter.BLOCKSCOPIED,
          getBlocks(srcstat.getLen(), srcstat.getBlockSize()));
      reporter.incrCounter(Counter.COPY, 1);
      updateStatus(reporter);
    }
    
    /** rename tmp to dst, delete dst if already exists */
    private void rename(Path tmp, Path dst) throws IOException {
      try {
        if (destFileSys.exists(dst)) {
          destFileSys.delete(dst, true);
        }
        if (!destFileSys.rename(tmp, dst)) {
          throw new IOException();
        }
      }
      catch(IOException cause) {
        throw (IOException)new IOException("Fail to rename tmp file (=" + tmp 
            + ") to destination file (=" + dst + ")").initCause(cause);
      }
    }

    private void updateDestStatus(FileStatus src, FileStatus dst
        ) throws IOException {
      if (preserve_status) {
        DistCp.updateDestStatus(src, dst, preserved, destFileSys);
      }
    }

    static String bytesString(long b) {
      return b + " bytes (" + StringUtils.humanReadableInt(b) + ")";
    }

    /** Mapper configuration.
     * Extracts source and destination file system, as well as
     * top-level paths on source and destination directories.
     * Gets the named file systems, to be used later in map.
     */
    public void configure(JobConf job)
    {
      destPath = new Path(job.get(DST_DIR_LABEL, "/"));
      tmpRoot = new Path(job.get(TMP_DIR_LABEL));
      String attemptId = job.get("mapred.task.id");
      attemptTmpRoot = new Path(tmpRoot, attemptId);

      try {
        destFileSys = destPath.getFileSystem(job);
      } catch (IOException ex) {
        throw new RuntimeException("Unable to get the named file system.", ex);
      }
      sizeBuf = job.getInt("copy.buf.size", 128 * 1024);
      buffer = new byte[sizeBuf];
      ignoreReadFailures = job.getBoolean(Options.IGNORE_READ_FAILURES.propertyname, false);
      preserve_status = job.getBoolean(Options.PRESERVE_STATUS.propertyname, false);
      if (preserve_status) {
        preserved = FileAttribute.parse(job.get(PRESERVE_STATUS_LABEL));
      }
      update = job.getBoolean(Options.UPDATE.propertyname, false);
      overwrite = !update && job.getBoolean(Options.OVERWRITE.propertyname, false);
      skipCRCCheck = job.getBoolean(Options.SKIPCRC.propertyname, false);
      useFastCopy = job.getBoolean(Options.USEFASTCOPY.propertyname, false);

      if (useFastCopy) {
        try {
          fc = new FastCopy(job);
        } catch (Exception e) {
          LOG.error("Exception during fastcopy instantiation", e);
        }
      }
      this.job = job;
    }

    /** Map method. Copies one file from source file system to destination.
     * @param key src len
     * @param value FilePair (FileStatus src, Path dst)
     * @param out Log of failed copies
     * @param reporter
     */
    public void map(LongWritable key,
                    FilePairComparable value,
                    OutputCollector<FilePairComparable, Text> out,
                    Reporter reporter) throws IOException {
      final FileStatus srcstat = value.input;
      final Path relativedst = new Path(value.output);
      try {
        copy(value, out, reporter);
      } catch (IOException e) {
        ++failcount;
        reporter.incrCounter(Counter.FAIL, 1);
        updateStatus(reporter);
        final String sfailure = "FAIL " + relativedst + " : " +
                          StringUtils.stringifyException(e);
        out.collect(value, new Text(sfailure));
        LOG.info(sfailure);
        try {
          for (int i = 0; i < 3; ++i) {
            try {
              final Path tmp = new Path(attemptTmpRoot, relativedst);
              if (destFileSys.delete(tmp, true))
                break;
            } catch (Throwable ex) {
              // ignore, we are just cleaning up
              LOG.debug("Ignoring cleanup exception", ex);
            }
            // update status, so we don't get timed out
            updateStatus(reporter);
            Thread.sleep(3 * 1000);
          }
        } catch (InterruptedException inte) {
          throw (IOException)new IOException().initCause(inte);
        }
      } finally {
        updateStatus(reporter);
      }
    }

    public void reduce(FilePairComparable file, Iterator<Text> statuses,
        OutputCollector<FilePairComparable, Text> out, Reporter reporter)
        throws IOException {
      final FileStatus srcStat = file.input;
      final Path relativeDest = new Path(file.output);
      Path absdst = new Path(destPath, relativeDest);
      Path finalTmpFile = new Path(tmpRoot, relativeDest);

      int totfiles = job.getInt(SRC_COUNT_LABEL, -1);
      assert totfiles >= 0 : "Invalid file count " + totfiles;

      if (totfiles == 1) {
        // Copying a single file; use dst path provided by user as destination
        // rather than destination directory, if a file
        Path dstparent = absdst.getParent();
        if (!(destFileSys.exists(dstparent) &&
              destFileSys.getFileStatus(dstparent).isDir())) {
          absdst = dstparent;
        }
      }
      if (!destFileSys.exists(finalTmpFile)) {
        // This is a rerun of a reducer and finalTmp has been moved into the
        // proper location already.
        // Another option is that the file has not been copied over in the first
        // place, but that was caught in the mapper
        LOG.info("Skipping " + absdst + " as it has been moved already");
        return;
      }
      if (destFileSys.exists(absdst) &&
          destFileSys.getFileStatus(absdst).isDir()) {
        throw new IOException(absdst + " is a directory");
      }
      if (!destFileSys.mkdirs(absdst.getParent())) {
        throw new IOException("Failed to create parent dir: " +
            absdst.getParent());
      }
      rename(finalTmpFile, absdst);
      
      FileStatus dststat = destFileSys.getFileStatus(absdst);
      if (dststat.getLen() != srcStat.getLen()) {
        destFileSys.delete(absdst, false);
        throw new IOException("File size not matched: copied "
            + bytesString(dststat.getLen()) + " to dst (=" + absdst 
            + ") but expected " + bytesString(srcStat.getLen()) 
            + " from " + srcStat.getPath());        
      }
      updateDestStatus(srcStat, dststat);
    }

    public void close() throws IOException {
      if (fc != null) {
        fc.shutdown();
      }
      if (0 == failcount || ignoreReadFailures) {
        return;
      }
      throw new IOException(getCountString());
    }
  }
  
  /**
   * FSCopyFilesTask: The mapper for copying files between FileSystems.
   */
  static class CopyFilesByChunkMapper 
      implements Mapper<LongWritable, FileChunkPair, WritableComparable<?>, Text> {
    // config
    private int sizeBuf = 128 * 1024;
    private FileSystem destFileSys = null;
    private boolean ignoreReadFailures;
    private boolean preserve_status;
    private EnumSet<FileAttribute> preserved;
    private boolean overwrite;
    private boolean update;
    private Path destPath = null;
    private byte[] buffer = null;
    private JobConf job;
    private boolean skipCRCCheck = false;

    // stats
    private int failcount = 0;
    private int skipcount = 0;
    private int copycount = 0;

    private String getCountString() {
      return "Copied: " + copycount + " Skipped: " + skipcount
      + " Failed: " + failcount;
    }
    private void updateStatus(Reporter reporter) {
      reporter.setStatus(getCountString());
    }

    /**
     * Return true if dst should be replaced by src and the update flag is set.
     * Right now, this merely checks that the src and dst len are not equal. 
     * This should be improved on once modification times, CRCs, etc. can
     * be meaningful in this context.
     * @throws IOException 
     */
    private boolean needsUpdate(FileStatus srcstatus,
        FileSystem dstfs, Path dstpath) throws IOException {
      return update && !sameFile(srcstatus.getPath().getFileSystem(job),
          srcstatus, dstfs, dstpath, skipCRCCheck);
    }

    private FSDataOutputStream create(Path f, Reporter reporter,
        FileStatus srcstat) throws IOException {
      if (destFileSys.exists(f)) {
        destFileSys.delete(f, false);
      }
      if (!preserve_status) {
        return destFileSys.create(f, true, sizeBuf, reporter);
      }

      FsPermission permission = preserved.contains(FileAttribute.PERMISSION)?
          srcstat.getPermission(): null;
      short replication = preserved.contains(FileAttribute.REPLICATION)?
          srcstat.getReplication(): destFileSys.getDefaultReplication();
      long blockSize = preserved.contains(FileAttribute.BLOCK_SIZE)?
          srcstat.getBlockSize(): destFileSys.getDefaultBlockSize();
      return destFileSys.create(f, permission, true, sizeBuf, replication,
          blockSize, reporter);
    }

    /**
     * Copy a file to a destination.
     * @param srcstat src path and metadata
     * @param relativedst dst path
     * @param offset the start point of the file chunk
     * @param length the length of the file chunk
     * @param chunkIndex the chunkIndex of the file chunk
     * @param reporter
     */
    private void copy(FileStatus srcstat, Path relativedst, long offset, long length,
        int chunkIndex, OutputCollector<WritableComparable<?>, Text> outc, 
        Reporter reporter) throws IOException {
      Path absdst = new Path(destPath, relativedst);
      int totfiles = job.getInt(SRC_COUNT_LABEL, -1);
      assert totfiles >= 0 : "Invalid file count " + totfiles;

      // if a directory, ensure created even if empty
      if (srcstat.isDir()) {
        if (destFileSys.exists(absdst)) {
          if (!destFileSys.getFileStatus(absdst).isDir()) {
            throw new IOException("Failed to mkdirs: " + absdst+" is a file.");
          }
        }
        else if (!destFileSys.mkdirs(absdst)) {
          throw new IOException("Failed to mkdirs " + absdst);
        }
        // TODO: when modification times can be set, directories should be
        // emitted to reducers so they might be preserved. Also, mkdirs does
        // not currently return an error when the directory already exists;
        // if this changes, all directory work might as well be done in reduce
        return;
      }

      //if a file
      //here skip count acctually counts how many file chunks are skipped
      if (destFileSys.exists(absdst) && !overwrite
          && !needsUpdate(srcstat, destFileSys, absdst)) {
        outc.collect(null, new Text("SKIP: " + srcstat.getPath()));
        ++skipcount;
        reporter.incrCounter(Counter.SKIP, 1);
        updateStatus(reporter);
        return;
      }


      Path tmpFile = new Path(job.get(TMP_DIR_LABEL),
          createFileChunkPath(relativedst, chunkIndex));
      long cbcopied = 0L;
      long needCopied = length;
      FSDataInputStream in = null;
      FSDataOutputStream out = null;
      try {
        // open src file
        in = srcstat.getPath().getFileSystem(job).open(srcstat.getPath());
        reporter.incrCounter(Counter.BYTESEXPECTED, length);
        // open tmp file
        out = create(tmpFile, reporter, srcstat);
        in.seek(offset);
        if(in.getPos() != offset){
          throw new IOException("File byte number doesn't match the offset.");
        }
        for(int cbread; (cbread = in.read(buffer)) >= 0; ) {
          if(needCopied == 0)
            break;
          if(needCopied >= cbread) {
            out.write(buffer, 0, cbread);
            cbcopied += cbread;
            needCopied -= cbread;
          }
          else {
            out.write(buffer, 0, (int) needCopied);
            cbcopied += needCopied;
            needCopied -= needCopied;
          }
          reporter.setStatus(
              String.format("%.2f ", cbcopied*100.0/length)
              + absdst + " [ " +
              StringUtils.humanReadableInt(cbcopied) + " / " +
              StringUtils.humanReadableInt(length) + " ]");
          if(needCopied == 0)
            break;
        }

      } finally {
        checkAndClose(in);
        checkAndClose(out);
      }

      if (cbcopied != length) {
        throw new IOException("File size not matched: copied "
            + bytesString(cbcopied) + " to tmpFile (=" + tmpFile
            + ") but expected " + bytesString(length) 
            + " from " + srcstat.getPath());        
      }
      else {
        FileStatus tmpstat = destFileSys.getFileStatus(tmpFile);
        updateDestStatus(srcstat, tmpstat);
      }

      // report at least once for each file chunk
      ++copycount;
      reporter.incrCounter(Counter.BYTESCOPIED, cbcopied);
      reporter.incrCounter(Counter.COPY, 1);
      updateStatus(reporter);

    }

    /**
     * create one directory for each file, call it filename_chunkfiles
     * and store the file chunks into that directory, name them by it's chunk chunkIndex
     * @return the path of the directory for that file
     */
    private Path createFileChunkPath(Path dst, int chunkIndex) 
        throws IOException{
      Path chunkFileDir = new Path(dst + "_chunkfiles");
      return new Path(chunkFileDir,Long.toString(chunkIndex));
    }

    private void updateDestStatus(FileStatus src, FileStatus dst
        ) throws IOException {
      if (preserve_status) {
        DistCp.updateDestStatus(src, dst, preserved, destFileSys);
      }
    }

    static String bytesString(long b) {
      return b + " bytes (" + StringUtils.humanReadableInt(b) + ")";
    }

    /** Mapper configuration.
     * Extracts source and destination file system, as well as
     * top-level paths on source and destination directories.
     * Gets the named file systems, to be used later in map.
     */
    public void configure(JobConf job)
    {
      destPath = new Path(job.get(DST_DIR_LABEL, "/"));
      try {
        destFileSys = destPath.getFileSystem(job);
      } catch (IOException ex) {
        throw new RuntimeException("Unable to get the named file system.", ex);
      }
      sizeBuf = job.getInt("copy.buf.size", 128 * 1024);
      buffer = new byte[sizeBuf];
      ignoreReadFailures = job.getBoolean(Options.IGNORE_READ_FAILURES.propertyname, false);
      preserve_status = job.getBoolean(Options.PRESERVE_STATUS.propertyname, false);
      if (preserve_status) {
        preserved = FileAttribute.parse(job.get(PRESERVE_STATUS_LABEL));
      }
      update = job.getBoolean(Options.UPDATE.propertyname, false);
      overwrite = !update && job.getBoolean(Options.OVERWRITE.propertyname, false);
      skipCRCCheck = job.getBoolean(Options.SKIPCRC.propertyname, false);
      this.job = job;
    }

    /** Map method. Copies one file from source file system to destination.
     * @param key src len
     * @param value FileChunkPair (FileStatus src, Path dst, long offset, 
     * long length, int chunkIndex)
     * @param out Log of failed copies
     * @param reporter
     */
    public void map(LongWritable key,
                    FileChunkPair value,
                    OutputCollector<WritableComparable<?>, Text> out,
                    Reporter reporter) throws IOException {
      final FileStatus srcstat = value.input;
      final Path relativedst = new Path(value.output);
      final long offset = value.offset;
      final long length = value.length;
      final int chunkIndex = value.chunkIndex;
      LOG.info(relativedst + " offset :"  + offset + " length : " + length + " chunkIndex : " + chunkIndex);
      try {
          copy(srcstat, relativedst, offset, length, chunkIndex, out, reporter);
      } catch (IOException e) {
        ++failcount;
        reporter.incrCounter(Counter.FAIL, 1);
        updateStatus(reporter);
        final String sfailure = "FAIL " + relativedst + " : " +
                          StringUtils.stringifyException(e);
        out.collect(null, new Text(sfailure));
        LOG.info(sfailure);
        try {
          for (int i = 0; i < 3; ++i) {
            try {
              final Path tmp = new Path(job.get(TMP_DIR_LABEL), relativedst);
              if (destFileSys.delete(tmp, true))
                break;
            } catch (Throwable ex) {
              // ignore, we are just cleaning up
              LOG.debug("Ignoring cleanup exception", ex);
            }
            // update status, so we don't get timed out
            updateStatus(reporter);
            Thread.sleep(3 * 1000);
          }
        } catch (InterruptedException inte) {
          throw (IOException)new IOException().initCause(inte);
        }
      } finally {
        updateStatus(reporter);
      }
    }

    public void close() throws IOException {
      if (0 == failcount || ignoreReadFailures) {
        return;
      }
      throw new IOException(getCountString());
    }
  }

  private static List<Path> fetchFileList(Configuration conf, Path srcList)
      throws IOException {
    List<Path> result = new ArrayList<Path>();
    FileSystem fs = srcList.getFileSystem(conf);
    BufferedReader input = null;
    try {
      input = new BufferedReader(new InputStreamReader(fs.open(srcList)));
      String line = input.readLine();
      while (line != null) {
        result.add(new Path(line));
        line = input.readLine();
      }
    } finally {
      checkAndClose(input);
    }
    return result;
  }

  @Deprecated
  public static void copy(Configuration conf, String srcPath,
                          String destPath, Path logPath,
                          boolean srcAsList, boolean ignoreReadFailures)
      throws IOException {
    final Path src = new Path(srcPath);
    List<Path> tmp = new ArrayList<Path>();
    if (srcAsList) {
      tmp.addAll(fetchFileList(conf, src));
    } else {
      tmp.add(src);
    }
    EnumSet<Options> flags = ignoreReadFailures
      ? EnumSet.of(Options.IGNORE_READ_FAILURES)
      : EnumSet.noneOf(Options.class);

    final Path dst = new Path(destPath);
    copy(conf, new Arguments(tmp, null, dst, logPath, flags, null,
        Long.MAX_VALUE, Long.MAX_VALUE, null));
  }

  /** Sanity check for srcPath */
  private static void checkSrcPath(Configuration conf, List<Path> srcPaths
      ) throws IOException {
    List<IOException> rslt = new ArrayList<IOException>();
    List<Path> unglobbed = new LinkedList<Path>();
    for (Path p : srcPaths) {
      FileSystem fs = p.getFileSystem(conf);
      FileStatus[] inputs = fs.globStatus(p);

      if(inputs.length > 0) {
        for (FileStatus onePath: inputs) {
          unglobbed.add(onePath.getPath());
        }
      } else {
        rslt.add(new IOException("Input source " + p + " does not exist."));
      }
    }
    if (!rslt.isEmpty()) {
      throw new InvalidInputException(rslt);
    }
    srcPaths.clear();
    srcPaths.addAll(unglobbed);
  }

  /**
   *  A copier class initialized for Distcp containing all the
   *  information needed for clients to launch the MapReduce job
   *  with method finalizeJob() to finalize the job and cleanupJob()
   *  to clean up the states.
   *
   *  Here is a typical pattern to use it:
   * 
   *     DistCopier copier = getCopier(conf, args);
   *    
   *     if (copier != null) {
   *       try {
   *         // launch job copier.getJobConf() and waits it to finish
   *         copier.finalizeCopiedFiles();
   *       } finally {
   *         copier.cleanupJob();
   *       }
   *     }
   */
  public static class DistCopier {
    private Configuration conf;
    private Arguments args;
    private JobConf jobToRun;
    private JobClient client;
    private boolean copyByChunk;
    
    /**
     * @return MapReduce job conf for the copying
     */
    public JobConf getJobConf() {
      return jobToRun;
    }

    public JobClient getJobClient() {
      return client;
    }
    
    /**
     * Finalize copied files after the MapReduce job. So far all it
     * does is to set metadata information of the copied files to be
     * the same as source files.
     * @throws IOException
     */
    public void finalizeCopiedFiles() throws IOException  {
      if (jobToRun != null) {
        finalizeCopiedFilesInternal(jobToRun);
      }
    }

    /**
     * Clean up temp files used for copying. It should be put
     * in finally block.
     * @throws IOException
     */
    public void cleanupJob()  throws IOException  {
      if (jobToRun != null) {
        cleanupJobInternal(jobToRun);
      }
    }

    private DistCopier(Configuration conf, Arguments args) throws IOException {
      this.conf = conf;
      this.args = args;
    }
    
    /**
     * @param srcs   source paths
     * @param dst    destination path
     * @param conf
     * @return True if, all source paths and destination path are DFS
     *         locations, and they are from the same DFS clusters. If
     *         it can't find the DFS cluster name since an older server
     *         build, it will assume cluster name matches.
     * @throws IOException
     */
    static public boolean canUseFastCopy(List<Path> srcs, Path dst,
        Configuration conf) throws IOException {
      DistributedFileSystem dstdfs = DFSUtil.convertToDFS(dst
          .getFileSystem(conf));
      if (dstdfs == null) {
        return false;
      }

      String dstClusterName = dstdfs.getClusterName();
      for (Path src : srcs) {
        DistributedFileSystem srcdfs = DFSUtil.convertToDFS(src
            .getFileSystem(conf));
        if (srcdfs == null) {
          return false;
        } else if (dstClusterName != null) {
          // We assume those clusterName == null case was older
          // version of DFS. We always enable fastcopy for those
          // cases.
          String srcClusterName = srcdfs.getClusterName();
          if (srcClusterName != null && !srcClusterName.equals(dstClusterName)) {
            return false;
          }
        }
      }
      return true;
    }

    private void setupJob() throws IOException  {
      LOG.info("srcPaths=" + args.srcs);
      LOG.info("destPath=" + args.dst);
      checkSrcPath(conf, args.srcs);
      JobConf job;
      copyByChunk = false;
      // if the -copybychunk flag is set, check if the file system allows using 
      // concat
      if(args.flags.contains(Options.COPYBYCHUNK)) {
        FileSystem dstfs = args.dst.getFileSystem(conf);
        //for raidDFS
        if(dstfs instanceof FilterFileSystem) {
            dstfs = ((FilterFileSystem) dstfs).getRawFileSystem();
        }
        if(dstfs instanceof DistributedFileSystem) {
          DistributedFileSystem dstdistfs = (DistributedFileSystem) dstfs;
          //set copybychunk to false if the concat method is not available for the
          //distributed file system
          DFSClient dfsClient = dstdistfs.getClient();
          if(dfsClient.isConcatAvailable())
            copyByChunk = true;
        }
        LOG.debug("After check, copy by chunk is set to: " + copyByChunk);
      }

      boolean useFastCopy = (args.flags.contains(Options.USEFASTCOPY) && canUseFastCopy(
          args.srcs, args.dst, conf));
      
      if(copyByChunk) {
        job = createJobConfForCopyByChunk(conf);
      } else {
        job = createJobConf(conf, useFastCopy);
      }
      if (args.preservedAttributes != null) {
        job.set(PRESERVE_STATUS_LABEL, args.preservedAttributes);
      }
      if (args.mapredSslConf != null) {
        job.set("dfs.https.client.keystore.resource", args.mapredSslConf);
      }

      try {
        try {
          if (client == null) {
            client = new JobClient(job);
          }
        } catch (IOException ex) {
          throw new IOException("Error creating JobClient", ex);
        }
        if(copyByChunk) {
          if (setupForCopyByChunk(conf, job, client, args)) {
            jobToRun = job;
          } else {
            finalizeCopiedFilesInternal(job);
          }
        } else {
          if (setup(conf, job, client, args, useFastCopy)) {
            jobToRun = job;
          } else {
            finalizeCopiedFilesInternal(job);
          }
        }
      } finally {
        if (jobToRun == null) {
          cleanupJobInternal(job);
        }
      }
    }
    
    private void finalizeCopiedFilesInternal(JobConf job) throws IOException {
      if (copyByChunk) {
        // if we copy the files by chunks, then need to stitch the file chunks
        // together back to the original file after the map
        LOG.debug("copy by chunk and stitch!");
        stitchChunks(conf, job, args);
      }
      DistCp.finalize(conf, job, args.dst, args.preservedAttributes);
    }

    private void cleanupJobInternal(JobConf job) throws IOException {
      //delete tmp
      fullyDelete(job.get(TMP_DIR_LABEL), job);
      //delete jobDirectory
      fullyDelete(job.get(JOB_DIR_LABEL), job);
    }
    
  }

  /**
   * Return a DistCopier object for copying the files.
   * 
   * If a MapReduce job is needed, a DistCopier instance will be
   * initialized and returned.
   * If no MapReduce job is needed (for empty directories), all the
   * other work will be done in this function, and NULL will be
   * returned. If caller sees NULL returned, the copying has
   * succeeded.
   * 
   * @param conf
   * @param args
   * @return
   * @throws IOException
   */
  public static DistCopier getCopier(final Configuration conf,
      final Arguments args) throws IOException {
    DistCopier dc = new DistCopier(conf, args);
    dc.setupJob();
    if (dc.getJobConf() != null) {
      return dc;
    } else {
      return null;
    }
  }

  /**
   * Driver to copy srcPath to destPath depending on required protocol.
   * @param args arguments
   */
  static void copy(final Configuration conf, final Arguments args
      ) throws IOException {
    DistCopier copier = getCopier(conf, args);
    
    if (copier != null) {
      try {
        JobClient client = copier.getJobClient();
        RunningJob job = client.submitJob(copier.getJobConf());
        try {
          if (!client.monitorAndPrintJob(copier.getJobConf(), job)) {
            throw new IOException("Job failed!");
          }
        } catch (InterruptedException ie) {
          Thread.currentThread().interrupt();
        }
        copier.finalizeCopiedFiles();
      } finally {
        copier.cleanupJob();
      }
    }
  }

  /**
   * In the dst file system, go through all the file chunks, stitch them
   * together to get back the original files.
   * After the mappers, file chunks are stored in the tmp directory.
   * 1. Read in the dst_chunk_file_dir_list, it contains key value pairs
   * such that key = num of chunk files within that directory, value = the 
   * original dst file name. 
   * For example, one entry in the list could be key = 3, value = /destdat/File1
   * 2. That entry tells us that after the mapper, we should have a directory
   * filename_chunkfiles in the tmp directory of the dst file system. 
   * Check whether that directory exist and check whether it contains 
   * number of file chunks that equals to the key value.
   * For example, we will check if there is a directory called
   * /destdat/tmp/File1_chunkfiles/, and if there are 3 files in that directory
   * 3. Concatenate the rest file chunks(if there is any) to the renamed 
   * chunk file 0
   * For examples:we will concatenate /destdat/tmp/File1_chunkfiles/1,
   * /destdat/tmp/File1_chunkfiles/2 to /destdat/tmp/File1_chunkfiles/0
   * 4. Rename the concatenated file
   * For examples: rename /destdat/tmp/File1_chunkfiles/0 to be
   * /destdat/File1
   * @throws IOException 
   */
  private static void stitchChunks(Configuration conf, JobConf jobConf,
      final Arguments args) throws IOException {
    //check if the file system is the dfs 
    FileSystem dstfs = args.dst.getFileSystem(conf);
    DistributedFileSystem dstdistfs = null;
    //for raidDFS, get the underlying filestems
    if(dstfs instanceof FilterFileSystem) {
      dstfs = ((FilterFileSystem) dstfs).getRawFileSystem(); 
    }
    if(dstfs instanceof DistributedFileSystem) {
      //cast dstfs to distributedFileSystem to use concat metho