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

import java.io.IOException;
import java.io.File;
import java.io.InterruptedIOException;
import java.util.Date;
import java.lang.Thread;
import java.net.InetSocketAddress;

import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.*;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FSImage;
import org.apache.hadoop.hdfs.server.namenode.AvatarNode;
import org.apache.hadoop.hdfs.tools.offlineImageViewer.LsImageVisitor;
import org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewer;
import org.apache.hadoop.hdfs.util.InjectionEvent;
import org.apache.hadoop.hdfs.util.InjectionHandler;
import org.apache.hadoop.http.HttpServer;

/**
 * This class drives the ingest of transaciton logs from primary.
 * It also implements periodic checkpointing of the  primary namenode.
 */

public class Standby implements Runnable{

  public static final Log LOG = AvatarNode.LOG;
  private static final long CHECKPOINT_DELAY = 10000; // 10 seconds
  private AvatarNode avatarNode;
  private Configuration confg; // configuration of local standby namenode
  private Configuration startupConf; // original configuration of AvatarNode
  private FSImage fsImage; // fsImage of the current namenode.
  private FSNamesystem fsnamesys; // fsnamesystem of the local standby namenode
  volatile private Ingest ingest;   // object that processes transaction logs from primary
  volatile private Thread ingestThread;  // thread that is procesing the transaction log
  volatile private boolean running;
  private final String machineName; // host name of name node

  //
  // These are for the Secondary NameNode.
  //
  private String fsName;                    // local namenode http name
  private InetSocketAddress nameNodeAddr;   // remote primary namenode address
  private NamenodeProtocol primaryNamenode; // remote primary namenode
  private HttpServer infoServer;
  private int infoPort;
  private String infoBindAddress;
  private long checkpointPeriod;        // in seconds
  private long checkpointSize;    // size (in MB) of current Edit Log
  private long lastCheckpointTime;
  private long earlyScheduledCheckpointTime = Long.MAX_VALUE;
  private long sleepBetweenErrors;
  private boolean checkpointEnabled;
  volatile private Thread backgroundThread;  // thread for secondary namenode 
  volatile private CheckpointSignature sig;
  private volatile String checkpointStatus;
  
  // two different types of ingested file
  public enum IngestFile { EDITS, EDITS_NEW };
  
  // allowed states of the ingest thread
  enum StandbyIngestState {
    NOT_INGESTING, 
    INGESTING_EDITS,
    QUIESCING_EDITS,
    CHECKPOINTING,
    INGESTING_EDITS_NEW,
    QUIESCING_EDITS_NEW,
    STANDBY_QUIESCED
  };
  
  // currently consumed ingest (edits, or edits.new)
  private volatile File currentIngestFile = null;
  protected volatile StandbyIngestState currentIngestState 
    = StandbyIngestState.NOT_INGESTING;
  protected Object ingestStateLock = new Object();
  private boolean lastFinalizeCheckpointFailed = false;
  
  // names of the edits files
  private final File editsFile;
  private final File editsFileNew;
  
  private final File tmpImageFileForValidation;

  // The Standby can either be processing transaction logs
  // from the primary namenode or it could be doing a checkpoint to upload a merged
  // fsimage to the primary.
  // The startupConf is the original configuration that was used to start the
  // AvatarNode. It is used by the secondary namenode to talk to the primary.
  // The "conf" is the configuration of the local standby namenode.
  //
  Standby(AvatarNode avatarNode, Configuration startupConf, Configuration conf) 
    throws IOException {
    this.running = true;
    this.avatarNode = avatarNode;
    this.confg = conf;
    this.startupConf = startupConf;
    this.fsImage = avatarNode.getFSImage();
    this.fsnamesys = avatarNode.getNamesystem();
    this.sleepBetweenErrors = startupConf.getInt("hdfs.avatarnode.sleep", 5000);
    initSecondary(startupConf); // start webserver for secondary namenode

    this.machineName =
      DNS.getDefaultHost(conf.get("dfs.namenode.dns.interface","default"),
                         conf.get("dfs.namenode.dns.nameserver","default"));
    LOG.info("machineName=" + machineName);
    
    this.editsFile = this.avatarNode.getRemoteEditsFile(conf);
    this.editsFileNew = this.avatarNode.getRemoteEditsFileNew(conf);
    
    InetSocketAddress addr = NameNode.getAddress(conf);
    this.tmpImageFileForValidation = new File("/tmp", 
        "hadoop_image." + addr.getHostName() + ":" + addr.getPort());
    checkpointStatus("No checkpoint initiated");
  }

  public void run() {
    backgroundThread = Thread.currentThread();
    while (running) {
      try {
        InjectionHandler.p