package org.apache.hadoop.hdfs;

import java.io.EOFException;
import java.io.IOException;
import java.io.FileNotFoundException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.atomic.AtomicInteger;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.OpenFileInfo;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlockWithMetaInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedBlocksWithMetaInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlockWithFileName;
import org.apache.hadoop.hdfs.protocol.LocatedDirectoryListing;
import org.apache.hadoop.hdfs.protocol.VersionedLocatedBlock;
import org.apache.hadoop.hdfs.protocol.VersionedLocatedBlocks;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.util.InjectionEvent;
import org.apache.hadoop.hdfs.util.InjectionHandler;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC.VersionIncompatible;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;

public class DistributedAvatarFileSystem extends DistributedFileSystem {

  static {
    Configuration.addDefaultResource("avatar-default.xml");
    Configuration.addDefaultResource("avatar-site.xml");
  }
  
  CachingAvatarZooKeeperClient zk;
  /*
   * ReadLock is acquired by the clients performing operations WriteLock is
   * acquired when we need to failover and modify the proxy. Read and write
   * because of read and write access to the namenode object.
   */
  ReentrantReadWriteLock fsLock = new ReentrantReadWriteLock(true);
  /**
   *  The canonical URI representing the cluster we are connecting to
   *  dfs1.data.xxx.com:9000 for example
   */
  URI logicalName;
  /**
   * The full address of the node in ZooKeeper
   */
  long lastPrimaryUpdate = 0;

  Configuration conf;
  // Wrapper for NameNodeProtocol that handles failover
  FailoverClientProtocol failoverClient;
  // Should DAFS retry write operations on failures or not
  boolean alwaysRetryWrites;
  // Indicates whether subscription model is used for ZK communication 
  boolean watchZK;
  boolean cacheZKData = false;
  // number of milliseconds to wait between successive attempts
  // to initialize standbyFS
  long standbyFSInitInterval;

  // time at which we last attempted to initialize standbyFS
  long lastStandbyFSInit = 0L;

  // number of milliseconds before we should check if a failover has occurred
  // (used when making calls to the standby avatar)
  long standbyFSCheckInterval;

  // time at which last check for failover was performed
  long lastStandbyFSCheck = 0L;

  // number of requests to standbyFS between checks for failover
  int standbyFSCheckRequestInterval;

  // number of requests to standbyFS since last last failover check
  AtomicInteger standbyFSCheckRequestCount = new AtomicInteger(0);

  volatile boolean shutdown = false;
  // indicates that the DFS is used instead of DAFS
  volatile boolean fallback = false;

  // We need to keep track of the FS object we used for failover
  DistributedFileSystem failoverFS;

  // a filesystem object that points to the standby avatar
  StandbyFS standbyFS = null;
  // URI of primary and standby avatar
  URI primaryURI;
  URI standbyURI;
  private int failoverCheckPeriod;

  // Will try for two minutes checking with ZK every 15 seconds
  // to see if the failover has happened in pull case
  // and just wait for two minutes in watch case
  public static final int FAILOVER_CHECK_PERIOD = 15000;
  public static final int FAILOVER_RETRIES = 8;
  // Tolerate up to 5 retries connecting to the NameNode
  private static final int FAILURE_RETRY = 5;

  /**
   * HA FileSystem initialization
   */

  @Override
  public URI getUri() {
    return this.logicalName;
  }

  public void initialize(URI name, Configuration conf) throws IOException {
    /*
     * If true clients holds a watch on the znode and acts on events If false -
     * failover is pull based. Client will call zookeeper exists()
     */
    watchZK = conf.getBoolean("fs.ha.zookeeper.watch", false);
    /*
     * If false - on Mutable call to the namenode we fail If true we try to make
     * the call go through by resolving conflicts
     */
    alwaysRetryWrites = conf.getBoolean("fs.ha.retrywrites", false);
    // The actual name of the filesystem e.g. dfs.data.xxx.com:9000
    this.logicalName = name;
    this.conf = conf;
    // Create AvatarZooKeeperClient
    Watcher watcher = null;
    if (watchZK) {
      watcher = new ZooKeeperFSWatcher();
    }
    zk = new CachingAvatarZooKeeperClient(conf, watcher);
    cacheZKData = zk.isCacheEnabled();
    // default interval between standbyFS initialization attempts is 10 mins
    standbyFSInitInterval = conf.getLong(&