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

import java.io.IOException;
import java.util.ArrayList;
import java.util.Formatter;
import java.util.List;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.hadoop.tools.rumen.TaskInfo;

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

/**
 * Synthetic job generated from a trace description.
 */
class GridmixJob implements Callable<Job>, Delayed {

  public static final String JOBNAME = "GRIDMIX";
  public static final String ORIGNAME = "gridmix.job.name.original";
  public static final Log LOG = LogFactory.getLog(GridmixJob.class);

  private static final ThreadLocal<Formatter> nameFormat =
    new ThreadLocal<Formatter>() {
      @Override
      protected Formatter initialValue() {
        final StringBuilder sb = new StringBuilder(JOBNAME.length() + 5);
        sb.append(JOBNAME);
        return new Formatter(sb);
      }
    };

  private final int seq;
  private final Path outdir;
  protected final Job job;
  private final JobStory jobdesc;
  private final long submissionTimeNanos;

  public GridmixJob(Configuration conf, long submissionMillis,
      JobStory jobdesc, Path outRoot, int seq) throws IOException {
    ((StringBuilder)nameFormat.get().out()).setLength(JOBNAME.length());
    job = new Job(conf, nameFormat.get().format("%05d", seq).toString());
    submissionTimeNanos = TimeUnit.NANOSECONDS.convert(
        submissionMillis, TimeUnit.MILLISECONDS);
    this.jobdesc = jobdesc;
    this.seq = seq;
    outdir = new Path(outRoot, "" + seq);
  }

  protected GridmixJob(Configuration conf, long submissionMillis, String name)
      throws IOException {
    job = new Job(conf, name);
    submissionTimeNanos = TimeUnit.NANOSECONDS.convert(
        submissionMillis, TimeUnit.MILLISECONDS);
    jobdesc = null;
    outdir = null;
    seq = -1;
  }

  public String toString() {
    return job.getJobName();
  }

  public long getDelay(TimeUnit unit) {
    return unit.convert(submissionTimeNanos - System.nanoTime(),
        TimeUnit.NANOSECONDS);
  }

  @Override
  public int compareTo(Delayed other) {
    if (this == other) {
      return 0;
    }
    if (other instanceof GridmixJob) {
      final long otherNanos = ((GridmixJob)other).submissionTimeNanos;
      if (otherNanos < submissionTimeNanos) {
        return 1;
      }
      if (otherNanos > submissionTimeNanos) {
        return -1;
      }
      return id() - ((GridmixJob)other).id();
    }
    final long diff =
      getDelay(TimeUnit.NANOSECONDS) - other.getDelay(TimeUnit.NANOSECONDS);
    return 0 == diff ? 0 : (diff > 0 ? 1 : -1);
  }

  @Override
  public boolean equals(Object other) {
    if (this == other) {
      return true;
    }
    // not possible unless job is cloned; all jobs should be unique
    return other instanceof GridmixJob && id() == ((GridmixJob)other).id();
  }

  @Override
  public int hashCode() {
    return id();
  }

  int id() {
    return seq;
  }

  Job getJob() {
    return job;
  }

  JobStory getJobDesc() {
    return jobdesc;
  }

  public Job call() throws IOException, InterruptedException,
                           ClassNotFoundException {
    job.setMapperClass(GridmixMapper.class);
    job.setReducerClass(GridmixReducer.class);
    job.setNumReduceTasks(jobdesc.getNumberReduces());
    job.setMapOutputKeyClass(GridmixKey.class);
    job.setMapOutputValueClass(GridmixRecord.class);
    job.setSortComparatorClass(GridmixKey.Comparator.class);
    job.setGroupingComparatorClass(SpecGroupingComparator.class);
    job.setInputFormatClass(GridmixInputFormat.class);
    job.setOutputFormatClass(RawBytesOutputFormat.class);
    job.setPartitionerClass(DraftPartitioner.class);
    job.setJarByClass(GridmixJob.class);
    job.getConfiguration().setInt("gridmix.job.seq", seq);
    job.ge