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

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.EOFException;
import java.io.PrintStream;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.StringTokenizer;
import java.util.ArrayList;
import java.util.List;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;

import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.util.LineReader;

import org.apache.hadoop.conf.Configured;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataInputStream;

import org.apache.hadoop.io.Text;

import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.Decompressor;

import org.codehaus.jackson.JsonEncoding;
import org.codehaus.jackson.JsonFactory;
import org.codehaus.jackson.JsonGenerator;
import org.codehaus.jackson.JsonProcessingException;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.map.SerializationConfig;

/**
 * This is the main class for rumen log mining functionality.
 * 
 * It reads a directory of job tracker logs, and computes various information
 * about it. See {@code usage()}, below.
 * 
 */
public class HadoopLogsAnalyzer extends Configured implements Tool {

  // output streams
  private PrintStream statusOutput = System.out;
  private PrintStream statisticalOutput = System.out;

  private static PrintStream staticDebugOutput = System.err;

  /**
   * The number of splits a task can have, before we ignore them all.
   */
  private final static int MAXIMUM_PREFERRED_LOCATIONS = 25;

  /**
   * This element is to compensate for the fact that our percentiles engine
   * rounds up for the expected sample count, so if the total number of readings
   * is small enough we need to compensate slightly when aggregating the spread
   * data from jobs with few reducers together with jobs with many reducers.
   */
  private static final long SMALL_SPREAD_COMPENSATION_THRESHOLD = 5L;

  /**
   * {@code MAXIMUM_CLOCK_SKEW} is the maximum plausible difference between the
   * clocks of machines in the same cluster. This is important because an event
   * that logically must follow a second event will be considered non-anomalous
   * if it precedes that second event, provided they happen on different
   * machines.
   */
  private static final long MAXIMUM_CLOCK_SKEW = 10000L;

  /**
   * The regular expression used to parse task attempt IDs in job tracker logs
   */
  private final static Pattern taskAttemptIDPattern = Pattern
      .compile(".*_([0-9]+)");

  private final static Pattern xmlFilePrefix = Pattern.compile("[ \t]*<");

  private final static Pattern confFileHeader = Pattern.compile("_conf.xml!!");

  private final Map<String, Pattern> counterPatterns = new HashMap<String, Pattern>();

  /**
   * The unpaired job config file. Currently only used to glean the {@code -Xmx}
   * field of the JRE options
   */
  private ParsedConfigFile jobconf = null;

  /**
   * Set by {@code -omit-task-details}. If true, we <i>only</i> emit the job
   * digest [statistical info], not the detailed job trace.
   */
  private boolean omitTaskDetails = false;

  private JsonGenerator jobTraceGen = null;

  private boolean prettyprintTrace = true;

  private LoggedJob jobBeingTraced = null;

  private Map<String, LoggedTask> tasksInCurrentJob;

  private Map<String, LoggedTaskAttempt> attemptsInCurrentJob;

  private Histogram[] successfulMapAttemptTimes;
  private Histogram successfulReduceAttemptTimes;
  private Histogram[] failedMapAttemptTimes;
  private Histogram failedReduceAttemptTimes;
  private Histogram successfulNthMapperAttempts;
  private Histogram successfulNthReducerAttempts;
  private Histogram mapperLocality;

  static final private Log LOG = LogFactory.getLog(HadoopLogsAnalyzer.class);

  private int[] attemptTimesPercentiles;

  private JsonGenerator topologyGen = null;

  private HashSet<ParsedHost> allHosts = new HashSet<ParsedHost>();

  // number of ticks per second
  private boolean collecting = false;

  private long lineNumber = 0;

  private String rereadableLine = null;

  private String inputFilename;

  private boolean inputIsDirectory = false;

  private Path inputDirectoryPath = null;
  private String[] inputDirectoryFiles = null;

  private int inputDirectoryCursor = -1;

  private LineReader input = null;
  private CompressionCodec inputCodec = null;
  private Decompressor inputDecompressor = null;
  private Text inputLineText = new Text();

  private boolean debug = false;

  private int version = 0;

  private int numberBuckets = 99;

  private int spreadMin;

  private int spreadMax;

  private boolean spreading = false;