/**
 * 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.hdfs;

import java.io.EOFException;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocksWithMetaInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.tools.FastCopy;
import org.apache.hadoop.hdfs.tools.FastCopy.FastFileCopyRequest;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.tools.FastCopy.FastCopyFileStatus;

import org.junit.AfterClass;

import static org.junit.Assert.*;

public class FastCopySetupUtil {

  private static final Random random = new Random();
  protected static Configuration conf;
  protected static Configuration remoteConf;
  private static MiniDFSCluster cluster;
  protected static DistributedFileSystem fs;
  private static MiniDFSCluster remoteCluster;
  private static DistributedFileSystem remoteFs;
  private static boolean pass = true;
  private static RWThread rwThread;
  public static final int FILESIZE = 1024 * 5; // 5 KB
  private static Map<Integer, DataNode> dnMap = new HashMap<Integer, DataNode>();

  private static Log LOG = LogFactory.getLog(FastCopySetupUtil.class);
  public static final int BLOCK_SIZE = 1024;
  private static final byte[] buffer = new byte[BLOCK_SIZE];
  public static final int TMPFILESIZE = 2048;
  private static final byte[] fileBuffer = new byte[TMPFILESIZE];
  public static final int BYTES_PER_CHECKSUM = 512;
  public static final int COPIES = 5;
  private static String confFile = "build/test/extraconf/core-site.xml";
  private static final int softLeasePeriod = 3 * 1000; // 3 sec
  private static final int hardLeasePeriod = 5 * 1000; // 5 sec

  public static void setUpClass() throws Exception {
    // Require the complete file to be replicated before we return in unit
    // tests.
    setConf("dfs.replication.min", 3);

    // Lower the pending replication timeout to make sure if any of our blocks
    // timeout the unit test catches it.
    setConf("dfs.replication.pending.timeout.sec", 60);

    // Make sure we get multiple blocks.
    setConf("dfs.block.size", BLOCK_SIZE);
    setConf("io.bytes.per.checksum", BYTES_PER_CHECKSUM);

    // Set low soft and hard lease period.
    setConf(FSConstants.DFS_HARD_LEASE_KEY, hardLeasePeriod);
    setConf(FSConstants.DFS_SOFT_LEASE_KEY, softLeasePeriod);

    System.setProperty("test.build.data", "build/test/data1");
    cluster = new MiniDFSCluster(conf, 6, true, new String[] { "/r1", "/r2",
        "/r1", "/r2", "/r1", "/r2" }, new String[] { "h1", "h2", "h3", "h1",
        "h2", "h3" });

    for (DataNode dn : cluster.getDataNodes()) {
      dnMap.put(dn.getSelfAddr().getPort(), dn);
    }

    // Writing conf to disk so that the FastCopy tool picks it up.
    FileOutputStream out = new FileOutputStream(confFile);
    conf.writeXml(out);
    fs = (DistributedFileSystem) cluster.getFileSystem();

    System.setProperty("test.build.data", "build/test/data2");
    remoteCluster = new MiniDFSCluster(remoteConf, 6, true, new String[] {
        "/r1", "/r2", "/r1", "/r2", "/r1", "/r2" }, new String[] { "h1", "h2",
        "h3", "h1", "h2", "h3" });

    for (DataNode dn : remoteCluster.getDataNodes()) {
      dnMap.put(dn.getSelfAddr().getPort(), dn);
    }

    remoteFs = (DistributedFileSystem) remoteCluster.getFileSystem();
    random.nextBytes(fileBuffer);
    rwThread = new RWThread();
    rwThread.start();
  }

  private static void setConf(String name, int value) {
    conf.setInt(name, value);
    remoteConf.setInt(name, value);
  }

  @AfterClass
  public static void tearDownClass() throws Exception {
    rwThread.stopRW();
    rwThread.join();
    remoteFs.close();
    remoteCluster.shutdown();
    fs.close();
    cluster.shutdown();
    // Remove the extra conf file.
    new File(confFile).delete();
  }

  private static class RWThread extends Thread {
    private boolean flag = true;
    private byte[] tmpBuffer = new byte[TMPFILESIZE];

    public void run() {
      while (flag) {
        try {
          // Make sure we have no pendingReplicationBlocks
          pass = (0 == cluster.getNameNode().namesystem
              .getPendingReplicationBlocks());
          pass = (0 == remoteCluster.getNameNode().namesystem
              .get