package org.apache.hadoop.mapred;

import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.HashSet;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.examples.RandomWriter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.UtilsForTests;
import org.apache.hadoop.mapred.lib.IdentityMapper;
import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.net.Node;

import junit.framework.TestCase;

public class TestJobInProgress extends TestCase {
  static final Log LOG = LogFactory.getLog(TestJobInProgress.class);

  private MiniMRCluster mrCluster;

  private MiniDFSCluster dfsCluster;
  JobTracker jt;
  private static Path TEST_DIR = 
    new Path(System.getProperty("test.build.data","/tmp"), "jip-testing");
  private static int numSlaves = 4;

  public static class FailMapTaskJob extends MapReduceBase implements
      Mapper<LongWritable, Text, Text, IntWritable> {

    @Override
    public void map(LongWritable key, Text value,
        OutputCollector<Text, IntWritable> output, Reporter reporter)
        throws IOException {
      // reporter.incrCounter(TaskCounts.LaunchedTask, 1);
      try {
        Thread.sleep(1000);
      } catch (InterruptedException e) {
        throw new IllegalArgumentException("Interrupted MAP task");
      }
      throw new IllegalArgumentException("Failing MAP task");
    }
  }

  // Suppressing waring as we just need to write a failing reduce task job
  // We don't need to bother about the actual key value pairs which are passed.
  @SuppressWarnings("unchecked")
  public static class FailReduceTaskJob extends MapReduceBase implements
      Reducer {

    @Override
    public void reduce(Object key, Iterator values, OutputCollector output,
        Reporter reporter) throws IOException {
      // reporter.incrCounter(TaskCounts.LaunchedTask, 1);
      try {
        Thread.sleep(1000);
      } catch (InterruptedException e) {
        throw new IllegalArgumentException("Failing Reduce task");
      }
      throw new IllegalArgumentException("Failing Reduce task");
    }

  }

  @Override
  protected void setUp() throws Exception {
    // TODO Auto-generated method stub
    super.setUp();
    Configuration conf = new Configuration();
    dfsCluster = new MiniDFSCluster(conf, numSlaves, true, null);
    mrCluster = new MiniMRCluster(numSlaves, dfsCluster.getFileSystem()
        .getUri().toString(), 1);
    jt = mrCluster.getJobTrackerRunner().getJobTracker();
  }

  public void testPendingMapTaskCount() throws Exception {
    launchTask(FailMapTaskJob.class, IdentityReducer.class);
    checkTaskCounts();
  }
  
  public void testPendingReduceTaskCount() throws Exception {
    launchTask(IdentityMapper.class, FailReduceTaskJob.class);
    checkTaskCounts();
  }

  /**
   * Test if running tasks are correctly maintained for various types of jobs
   */
  private void testRunningTaskCount(boolean speculation, boolean locality)
  throws Exception {
    LOG.info("Testing running jobs with speculation : " + speculation 
             + ", locality : " + locality);
    // cleanup
    dfsCluster.getFileSystem().delete(TEST_DIR, true);
    
    final Path mapSignalFile = new Path(TEST_DIR, "map-signal");
    final Path redSignalFile = new Path(TEST_DIR, "reduce-signal");
    
    // configure a waiting job with 2 maps and 2 reducers
    JobConf job = 
      configure(UtilsForTests.WaitingMapper.class, IdentityReducer.class, 1, 1,
                locality);
    job.set(UtilsForTests.getTaskSignalParameter(true), mapSignalFile.toString());
    job.set(UtilsForTests.getTaskSignalParameter(false), redSignalFile.toString());
    
    // Disable slow-start for reduces since this maps don't complete 
    // in these test-cases...
    job.setFloat("mapred.reduce.slowstart.completed.maps", 0.0f);
    
    // test jobs with speculation
    job.setSpeculativeExecution(speculation);
    JobClient jc = new JobClient(job);
    RunningJob running = jc.submitJob(job);
    JobTracker jobtracker = mrCluster.getJobTrackerRunner().getJobTracker();
    JobInProgress jip = jobtracker.getJob(running.getID());
    LOG.info("Running job " + jip.getJobID());
    
    // wait
    LOG.info("Waiting for job " + jip.getJobID() + " to be ready");
    waitTillReady(jip, job);
    
    // check if the running structures are populated
    Set<TaskInProgress> uniqueTasks = new HashSet<TaskInProgress>();
    for (Map.Entry<Node, Set<TaskInProgress>> s : 
           jip.getRunningMapCache().entrySet()) {
      uniqueTasks.addAll(s.getValue());
    }
    
    // add non local map tasks
    uniqueTasks.addAll(jip.getNonLocalRunningMaps());
    
    assertEquals("Running map count doesnt match for jobs with speculation " 
                 + speculation + ", and locality " + locality,
                 jip.runningMaps(), uniqueTasks.size());

    assertEquals("Running reducer count doesnt match for jobs with speculation "
                 + speculation + ", and locality " + locality,
                 jip.runningReduces(), jip.getRunningReduces().size());
    
    // signal the tasks
    LOG.info("Signaling the tasks");
    UtilsForTests.signalTasks(dfsCluster, dfsCluster.getFileSystem(),
                              mapSignalFile.toString(), 
                              redSignalFile.toString(), numSlaves);
    
    // wait for the job to complete
    LOG.info("Waiting for job " + jip.getJobID() + " to be complete");
    UtilsForTests.waitTillDone(jc);
    
    // cleanup
    dfsCluster.getFileSystem().delete(TEST_DIR, true);
  }
  
  // wait for the