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

import java.io.BufferedOutputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import java.util.List;
import java.util.Collection;
import java.util.regex.Pattern;
import java.util.Random;
import java.util.Queue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.channels.SocketChannel;

import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSInputStream;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.BlockSender;
import org.apache.hadoop.hdfs.server.datanode.FSDataset;
import org.apache.hadoop.io.Text;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockMissingException;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ipc.RPC;

import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ProtocolCompatible;
import org.apache.hadoop.hdfs.protocol.HighTideProtocol;
import org.apache.hadoop.hdfs.protocol.PolicyInfo;
import org.apache.hadoop.hdfs.protocol.PolicyInfo.PathInfo;
import org.apache.hadoop.hdfs.server.hightidenode.metrics.HighTideNodeMetrics;

/**
 * This class fixes files by copying data from one of the files in the
 * equivalent set.
 * It periodically fetches the list of corrupt files from the namenode,
 * and fixed missing blocks
 */
public class FileFixer implements Runnable {
  public static final Log LOG = LogFactory.getLog(
                                  "org.apache.hadoop.hdfs.hightide.FileFixer");
  private final Configuration conf;

  private volatile boolean running = true;
  private int blockFixInterval = 60*1000; // 1min
  private int numThreads = 100;

  // ThreadPool keep-alive time for threads over core pool size
  private static final long THREADS_KEEP_ALIVE_SECONDS = 60;

  // a queue to store corrupted files
  static class PathToPolicy {
    String spath;
    PolicyInfo pinfo;
    PathToPolicy(Path p, PolicyInfo info) {
      this.spath = p.toString();
      this.pinfo = info;
    }
  }

  private Collection<PolicyInfo> all; // list of all policies
  List<PathToPolicy> pathToPolicy;    // find policy based on longest path match

  private PendingReplication filesBeingFixed;  // files that are being fixed


  ThreadPoolExecutor executor;         // threads to fix blocks

  FileFixer(Configuration conf) throws IOException {
    this.conf = conf;
    blockFixInterval = conf.getInt("hightide.blockfix.interval",
                                   blockFixInterval);
    numThreads = conf.getInt("hightide.blockfix.numthreads", numThreads);

    pathToPolicy = new LinkedList<PathToPolicy>();
    executor = new ThreadPoolExecutor( numThreads, numThreads,
          THREADS_KEEP_ALIVE_SECONDS, TimeUnit.SECONDS,
          new LinkedBlockingQueue<Runnable>());

   // start a thread to purge enties from this set automatically
   filesBeingFixed = new PendingReplication(conf.getInt(
                           "dfs.hightide.pending.timeout.sec", -1) * 1000L);
  }

  /**
   * The list of all configured policies.
   */
  void setPolicyInfo(Collection<PolicyInfo> all) throws IOException {
    this.all = all;
    this.pathToPolicy.clear();

    // keep a reverse map from all top-level paths to policies
    for (PolicyInfo pinfo: all) {
      pathToPolicy.add(new PathToPolicy(pinfo.getSrcPath(), pinfo));
      for (PathInfo d:pinfo.getDestPaths()) {
        pathToPolicy.add(new PathToPolicy(d.rpath, pinfo));
      }
    }

    // keep all paths sorted in revere lexicographical order so that 
    // we longest path is first.
    Comparator<PathToPolicy> comp = new Comparator<PathToPolicy>() {
      public int compare(PathToPolicy p