Kaynağa Gözat

HADOOP-858. Move contrib/smallJobsBenchmark to src/test. Contributed by Nigel.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@504682 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 yıl önce
ebeveyn
işleme
b96a4fd94e

+ 3 - 0
CHANGES.txt

@@ -9,6 +9,9 @@ Trunk (unreleased changes)
  2. HADOOP-982.  Add some setters and a toString() method to
     BytesWritable.  (omalley via cutting)
 
+ 3. HADOOP-858.  Move contrib/smallJobsBenchmark to src/test, removing
+    obsolete bits. (Nigel Daley via cutting)
+
 
 Branch 0.11 - unreleased
 

+ 0 - 2
build.xml

@@ -493,7 +493,6 @@
     	<packageset dir="${examples.dir}"/>
 
     	<packageset dir="src/contrib/streaming/src/java"/>
-    	<packageset dir="src/contrib/smallJobsBenchmark/src/java"/>
     	<packageset dir="src/contrib/abacus/src/java"/>
 
         <link href="${javadoc.link.java}"/>
@@ -503,7 +502,6 @@
     	<group title="Examples" packages="org.apache.hadoop.examples*"/>
 
        <group title="contrib: Streaming" packages="org.apache.hadoop.streaming*"/>
-       <group title="contrib: Small Jobs Benchmark" packages="org.apache.hadoop.benchmarks.mapred*"/>
        <group title="contrib: Abacus" packages="org.apache.hadoop.abacus*"/>
 
     </javadoc>

+ 0 - 43
src/contrib/smallJobsBenchmark/Readme.txt

@@ -1,43 +0,0 @@
-SmallJobsBenchmark Readme : 
-
-Building the benchmark. 
-to build - 
-$ cd smallJobsBenchmark
-$ ant deploy
-
-Running the benchmark
-$ $HADOOP_HOME/bin/hadoop jar <path to MRBenchmark.jar> smallJobsBenchmark -inputLines <numLines> -maps <numMaps> -reduces <numReduces> -jar <path to MRBenchmark.jar>
-
-For running the benchmark multiple times with different input sizes, maps, reduces following helper scripts can be used. 
-$ cd build/contrib/smallJobsBenchmark
-$ bin/run.sh
-
-after successfully running the benchmark see logs/report.txt for consolidated output of all the runs. 
-
-change this script to configure options. 
-
-Configurable options are - 
-
--inputLines noOfLines 
-  no of lines of input to generate. 
-
--inputType (ascending, descending, random)
-  type of input to generate. 
-
--jar jarFilePath 
-  Jar file containing Mapper and Reducer implementations in jar file. By default ant build creates MRBenchmark.jar file containing default Mapper and Reducer. 
-  
--times numJobs 
-No of times to run each MapReduce task, time is calculated as average of all runs. 
-
--workDir dfsPath 
-DFS path to put output of MR tasks. 
-
--maps numMaps 
-No of maps for wach task 
-
--reduces numReduces 
-No of reduces for each task
-
--ignoreOutput
-Doesn't copy the output back to local disk. Otherwise it creates the output back to a temp location on local disk. 

+ 0 - 10
src/contrib/smallJobsBenchmark/bin/report.sh

@@ -1,10 +0,0 @@
-#!/bin/bash
-
-echo "DataLines, Maps, Reduces, AvgTime " > logs/report.txt
-for logFile in `ls logs/*.log`
-do
-#       tail -n $((${TIMES}+5))  ${logFile} >> logs/report.txt
-        tail -n 1  ${logFile} >> logs/report.txt
-done
-
-cat  logs/report.txt

+ 0 - 27
src/contrib/smallJobsBenchmark/bin/run.sh

@@ -1,27 +0,0 @@
-#!/bin/bash
-
-if [ -z $HADOOP_HOME ] 
-then
-  echo "Error HADOOP_HOME not defined"  ;
-  exit 1;
-fi
-
-mkdir -p logs;
-
-export TIMES=2
-
-#for dataLines in 1 10000 10000000 
-for dataLines in 1 100
- do 
-
-for maps in 1 18
-	do 
-	for reduces in 1 18
-	do
-$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/../contrib/smallJobsBenchmark/MRBenchmark.jar smallJobsBenchmark -inputLines ${dataLines} -output /hadoop/mapred/MROutput -jar $HADOOP_HOME/../contrib/smallJobsBenchmark/MRBenchmark.jar -times ${TIMES} -workDir /hadoop/mapred/work -maps ${maps} -reduces ${reduces} -inputType ascending  -ignoreOutput  2>&1 | tee logs/benchmark_${dataLines}_${maps}_${reduces}.log
-
-	done
-	done
-	done
-
-bin/report.sh

+ 0 - 31
src/contrib/smallJobsBenchmark/build.xml

@@ -1,31 +0,0 @@
-<?xml version="1.0"?>
-
-<project name="smallJobsBenchmark" default="deploy">
-
-  <import file="../build-contrib.xml"/>
-  
-  <!-- Same name is used by shell scripts running this 
-       benchmark -->
-  <property name="benchmarkJarName" value="MRBenchmark.jar"/>
-    
-  <target name="jar" depends="compile">
-    <jar
-      jarfile="${build.dir}/${benchmarkJarName}"
-      basedir="${build.classes}">
-    <manifest>
-       <attribute name="Main-Class" 
-                  value="org/apache/hadoop/benchmarks/mapred/BenchmarkRunner"/>
-     </manifest>
-    </jar>
-  </target>
-  
-  <target name="deploy" depends="jar">
-    <mkdir dir="${build.dir}/bin"/>
-    <copy todir="${build.dir}/bin">
-      <fileset dir="${root}/bin">
-        <include name="**/*.*"/>
-      </fileset>
-    </copy> 
-  </target>  
-  
-  </project>

+ 0 - 34
src/contrib/smallJobsBenchmark/src/java/org/apache/hadoop/benchmarks/mapred/BenchmarkMapper.java

@@ -1,34 +0,0 @@
-package org.apache.hadoop.benchmarks.mapred;
-
-import java.io.IOException;
-import java.io.StringWriter;
-
-import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * takes inpt format as text lines, runs some processing on it and 
- * writes out data as text again. 
- * 
- * @author sanjaydahiya
- *
- */
-public class BenchmarkMapper extends MapReduceBase implements Mapper {
-  
-  public void map(WritableComparable key, Writable value,
-      OutputCollector output, Reporter reporter) throws IOException {
-    
-    String line = value.toString();
-    output.collect(new UTF8(process(line)), new UTF8(""));		
-  }
-  
-  public String process(String line){
-    return line ; 
-  }
-  
-}

+ 0 - 32
src/contrib/smallJobsBenchmark/src/java/org/apache/hadoop/benchmarks/mapred/BenchmarkReducer.java

@@ -1,32 +0,0 @@
-package org.apache.hadoop.benchmarks.mapred;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * 
- * @author sanjaydahiya
- *
- */
-public class BenchmarkReducer extends MapReduceBase implements Reducer {
-  
-  public void reduce(WritableComparable key, Iterator values,
-      OutputCollector output, Reporter reporter) throws IOException {
-    
-    // ignore the key and write values to output
-    while(values.hasNext()){
-      output.collect(key, new UTF8(values.next().toString()));
-    }
-  }
-  
-  public String process(String line){
-    return line ;
-  }
-}

+ 0 - 23
src/contrib/smallJobsBenchmark/src/java/org/apache/hadoop/benchmarks/mapred/BenchmarkRunner.java

@@ -1,23 +0,0 @@
-package org.apache.hadoop.benchmarks.mapred;
-
-
-import org.apache.hadoop.util.ProgramDriver;
-/**
- * Driver for benchmark. 
- * @author sanjaydahiya
- *
- */
-public class BenchmarkRunner {
-  
-  public static void main(String argv[]){
-    ProgramDriver pgd = new ProgramDriver();
-    try {
-      pgd.addClass("smallJobsBenchmark", MultiJobRunner.class, 
-      "A map/reduce benchmark that creates many small jobs");
-      pgd.driver(argv);
-    }
-    catch(Throwable e){
-      e.printStackTrace();
-    }
-  }
-}

+ 0 - 64
src/contrib/smallJobsBenchmark/src/java/org/apache/hadoop/benchmarks/mapred/GenData.java

@@ -1,64 +0,0 @@
-package org.apache.hadoop.benchmarks.mapred;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Random;
-
-
-public class GenData {
-  public static final int RANDOM = 1; 
-  public static final int ASCENDING = 2; 
-  public static final int DESCENDING = 3; 
-  
-  public static void generateText(long numLines, File file, int sortType) throws IOException{
-    
-    PrintStream output = new PrintStream(new FileOutputStream(file)); 
-    int padding = String.valueOf(numLines).length();
-    
-    switch(sortType){
-    
-    case RANDOM : 
-      for(long l = 0 ; l<numLines ; l++ ){
-        output.println(pad((new Random()).nextLong(), padding));
-      }
-      break ; 
-      
-    case ASCENDING: 
-      for(long l = 0 ; l<numLines ; l++ ){
-        output.println(pad(l, padding));
-      }
-      break ;
-      
-    case DESCENDING: 
-      for(long l = numLines ; l>0 ; l-- ){
-        output.println(pad(l, padding));
-      }
-      break ;
-      
-    }
-    output.close() ; 
-  }
-  
-  private static String pad( long number, int size ){
-    String str = String.valueOf(number);
-    
-    StringBuffer value = new StringBuffer(); 
-    for( int i = str.length(); i< size ; i++ ){
-      value.append("0"); 
-    }
-    value.append(str); 
-    return value.toString();
-  }
-  
-  public static void main(String[] args){
-    try{
-      // test 
-      generateText(100, new File("/Users/sanjaydahiya/dev/temp/sort.txt"), ASCENDING);
-    }catch(Exception e){
-      e.printStackTrace();
-    }
-  }
-  
-}

+ 0 - 407
src/contrib/smallJobsBenchmark/src/java/org/apache/hadoop/benchmarks/mapred/MultiJobRunner.java

@@ -1,407 +0,0 @@
-package org.apache.hadoop.benchmarks.mapred;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-
-/**
- * Runs a job multiple times and takesaverage of all runs. 
- * @author sanjaydahiya
- *
- */
-public class MultiJobRunner {
-  
-  private String jarFile = null ; // "MRBenchmark.jar" ;
-  private String input ; 
-  private String output ; 
-  private int numJobs = 1 ; // default value
-  private static final Log LOG = LogFactory.getLog(MultiJobRunner.class);
-  private int numMaps = 2; 
-  private int numReduces = 1;
-  private int dataLines = 1; 
-  private boolean ignoreOutput = false ; 
-  private boolean verbose = false ; 
-  
-  // just to print in the end
-  ArrayList execTimes = new ArrayList(); 
-  
-  private static String context = "/mapred/benchmark"; 
-  
-  /**
-   * Input is a local file. 
-   * @param input
-   * @param output
-   * @param jarFile
-   */
-  public MultiJobRunner(String input, String output, String jarFile){
-    this.input = input ; 
-    this.output = output ; 
-    this.jarFile = jarFile ; 
-  }
-  
-  public String getInput() {
-    return input;
-  }
-  
-  public void setInput(String input) {
-    this.input = input;
-  } 
-  
-  public String getJarFile() {
-    return jarFile;
-  }
-  
-  public void setJarFile(String jarFile) {
-    this.jarFile = jarFile;
-  }
-  
-  public String getOutput() {
-    return output;
-  }
-  
-  public void setOutput(String output) {
-    this.output = output;
-  }
-  
-  public int getNumJobs() {
-    return numJobs;
-  }
-  
-  public void setNumJobs(int numJobs) {
-    this.numJobs = numJobs;
-  }
-  
-  
-  public int getDataLines() {
-    return dataLines;
-  }
-  
-  public void setDataLines(int dataLines) {
-    this.dataLines = dataLines;
-  }
-  
-  public boolean isIgnoreOutput(){
-    return this.ignoreOutput ; 
-  }
-  
-  public void setIgnoreOutput(boolean ignore){
-    this.ignoreOutput = ignore ; 
-  }
-  
-  public void setVerbose(boolean verbose){
-    this.verbose = verbose ; 
-  }
-  public boolean getVerbose(){
-    return this.verbose; 
-  }
-  
-  /**
-   * Prepare the jobConf.
-   * @return
-   */
-  private JobConf setupJob(){
-    JobConf job = new JobConf() ;
-    
-    job.addInputPath(new Path(context + "/input"));
-    
-    job.setInputFormat(TextInputFormat.class);
-    job.setOutputFormat(TextOutputFormat.class);
-    
-    job.setInputKeyClass(LongWritable.class);
-    job.setOutputValueClass(UTF8.class);
-    
-    job.setMapOutputKeyClass(UTF8.class);
-    job.setMapOutputValueClass(UTF8.class);
-    
-    job.setOutputPath(new Path(output));
-    
-    if( null != jarFile ){
-      job.setJar(jarFile);
-    }
-    job.setMapperClass(BenchmarkMapper.class);
-    job.setReducerClass(BenchmarkReducer.class);
-    
-    job.setNumMapTasks(this.numMaps);
-    job.setNumReduceTasks(this.numReduces);
-    
-    return job ; 
-  }
-  
-  /**
-   * Runs a MapReduce task, given number of times. The input to each task is the same file. 
-   * @param job
-   * @param times
-   * @throws IOException
-   */
-  private void runJobInSequence(int times) throws IOException{
-    Path intrimData = null ; 
-    Random rand = new Random();
-    
-    for( int i= 0;i<times;i++){
-      // create a new job conf every time, reusing same object doesnt seem to work. 
-      JobConf job = setupJob();
-      
-      // give a new random name to output of the mapred tasks
-      // TODO: see if something better can be done
-      intrimData = new Path(context+"/temp/multiMapRedOutput_" + 
-          rand.nextInt() );
-      job.setOutputPath(intrimData);
-      
-      // run the mapred task now 
-      LOG.info("Running job, Input : " + job.getInputPaths()[0] + 
-          " Output : " + job.getOutputPath());
-      long curTime = System.currentTimeMillis();
-      JobClient.runJob(job);
-      execTimes.add(new Long(System.currentTimeMillis() - curTime));
-      
-      // pull the output out of DFS for validation
-      File localOutputFile = File.createTempFile("MROutput" + 
-          new Random().nextInt(), ".txt" );
-      String localOutputPath = localOutputFile.getAbsolutePath() ; 
-      localOutputFile.delete(); 
-      
-      if( ! ignoreOutput ){
-        copyFromDFS(intrimData, localOutputPath);
-      }
-      
-      // diff(input, localOutputPath);
-    }
-  }
-  
-  /**
-   * Not using it. 
-   */
-  private boolean diff(String path1, String path2) throws IOException{
-    boolean ret = false ; 
-    
-    return ret ; 
-  }
-  
-  /**
-   * Runs a sequence of map reduce tasks, output of each reduce is input 
-   * to next map. input should be a pre configured array of JobConfs. 
-   * 
-   */
-  public Path runJobsInSequence(JobConf[] jobs) throws IOException{
-    
-    // input location = jobs[0] input loc
-    Path finalOutput = null ; 
-    
-    for( int i=0;i<jobs.length; i++){
-      if( 0 != i ) {
-        // run the first job in sequence. 
-        jobs[i].addInputPath(finalOutput) ; 
-      }
-      
-      JobClient.runJob(jobs[i]);
-      finalOutput = jobs[i].getOutputPath(); 
-    }
-    
-    return finalOutput; 
-  }
-  
-  /**
-   * 
-   * Copy the input file from local disk to DFS. 
-   * @param localFile
-   * @param remotePath
-   * @return
-   * @throws IOException
-   */
-  private Path copyToDFS(String localFile, Path remotePath) throws IOException{
-    if( null == remotePath){ 
-      // use temp path under /mapred in DFS
-      remotePath =  new Path( context+"/input/MRBenchmark_" + 
-          new Random().nextInt()) ;
-    }
-    //new File(localPath).
-    Configuration conf = new Configuration();
-    FileSystem localFS = FileSystem.getLocal(conf);
-    FileSystem remoteFS = FileSystem.get(conf);
-    
-    FileUtil.copy(localFS, new Path(localFile), remoteFS, 
-        remotePath, false, conf);
-    
-    if( ignoreOutput) {
-      // delete local copy 
-      new File(localFile).delete();
-    }
-    
-    return remotePath; 
-  }
-  
-  private void copyFromDFS(Path remotePath, String localPath)
-  throws IOException{
-    
-    Configuration conf = new Configuration();
-    FileSystem localFS = FileSystem.getLocal(conf);
-    FileSystem remoteFS = FileSystem.get(conf);
-    
-    FileUtil.copy(remoteFS, remotePath, 
-        localFS, new Path(localPath), false, conf);
-  }
-  
-  private void setupContext() throws IOException{
-    FileSystem.get(new Configuration()).mkdirs(new Path(context));
-  }
-  private void clearContext() throws IOException{
-    FileSystem.get(new Configuration()).delete(new Path(context));
-  }
-  /**
-   * Run the benchmark. 
-   * @throws IOException
-   */
-  public void run() throws IOException{
-    
-    setupContext(); 
-    Path path = copyToDFS(input, null);
-    
-    long time = System.currentTimeMillis();
-    
-    try{
-      runJobInSequence(numJobs);
-    }finally{
-      clearContext(); 
-    }
-    
-    if( verbose ) {
-      // Print out a report 
-      System.out.println("Total MapReduce tasks executed: " + this.numJobs);
-      System.out.println("Total lines of data : " + this.dataLines);
-      System.out.println("Maps : " + this.numMaps + 
-          " ,  Reduces : " + this.numReduces);
-    }
-    int i =0 ; 
-    long totalTime = 0 ; 
-    for( Iterator iter = execTimes.iterator() ; iter.hasNext();){
-      totalTime +=  ((Long)iter.next()).longValue() ; 
-      if( verbose ) {
-        System.out.println("Total time for task : " + ++i + 
-            " , =  " +  (Long)iter.next());
-      }
-    }
-    
-    long avgTime = totalTime / numJobs ;
-    if( verbose ) {
-      System.out.println("Avg time : " + avgTime);
-    }
-    
-    System.out.println("DataLines  Maps    Reduces    AvgTime");
-    System.out.println(this.dataLines + ", " + this.numMaps + ", " + 
-        this.numReduces + ", " + avgTime);
-    
-  }
-  
-  public int getNumMaps() {
-    return numMaps;
-  }
-  
-  public void setNumMaps(int numMaps) {
-    this.numMaps = numMaps;
-  }
-  
-  public int getNumReduces() {
-    return numReduces;
-  }
-  
-  public void setNumReduces(int numReduces) {
-    this.numReduces = numReduces;
-  }
-  
-  public static void main (String[] args) throws IOException{
-    
-    String version = "MRBenchmark.0.0.1";
-    String usage = 
-      "Usage: MultiJobRunner -inputLines noOfLines -jar jarFilePath " + 
-      "[-output dfsPath] [-times numJobs] -workDir dfsPath" +  
-      "[-inputType (ascending, descending, random)]" + 
-      " -maps numMaps -reduces numReduces -ignoreOutput -verbose" ;
-    
-    System.out.println(version);
-    
-    if (args.length == 0) {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    
-    String output = "";
-    String jarFile = null; //"MRBenchmark.jar" ; 
-    int numJobs = 1 ; 
-    int numMaps = 2; 
-    int numReduces = 1 ; 
-    int dataLines = 1 ; 
-    int inputType = GenData.ASCENDING ; 
-    boolean ignoreOutput = false ; 
-    boolean verbose = false ; 
-    
-    for (int i = 0; i < args.length; i++) { // parse command line
-      if (args[i].equals("-output")) {
-        output = args[++i];
-      }else if (args[i].equals("-jar")) {
-        jarFile = args[++i];
-      }else if (args[i].equals("-times")) {
-        numJobs = Integer.parseInt(args[++i]);
-      }else if(args[i].equals("-workDir")) {
-        context = args[++i];
-      }else if(args[i].equals("-maps")) {
-        numMaps = Integer.parseInt(args[++i]);
-      }else if(args[i].equals("-reduces")) {
-        numReduces = Integer.parseInt(args[++i]);
-      }else if(args[i].equals("-inputLines")) {
-        dataLines = Integer.parseInt(args[++i]);
-      }else if(args[i].equals("-inputType")) {
-        String s = args[++i] ; 
-        if( s.equals("ascending")){
-          inputType = GenData.ASCENDING ;
-        }else if(s.equals("descending")){
-          inputType = GenData.DESCENDING ; 
-        }else if(s.equals("random")){
-          inputType = GenData.RANDOM ;
-        }
-      }else if(args[i].equals("-ignoreOutput")) {
-        ignoreOutput = true ;
-      }else if(args[i].equals("-verbose")) {
-        verbose = true ;
-      }
-    }
-    
-    File inputFile = File.createTempFile("SortedInput_" + 
-        new Random().nextInt(),".txt" );
-    GenData.generateText(dataLines, inputFile, inputType);
-    
-    MultiJobRunner runner = new MultiJobRunner(inputFile.getAbsolutePath(), 
-        output, jarFile );
-    runner.setInput(inputFile.getAbsolutePath());
-    runner.setNumMaps(numMaps);
-    runner.setNumReduces(numReduces);
-    runner.setDataLines(dataLines);
-    runner.setIgnoreOutput(ignoreOutput);
-    runner.setVerbose(verbose);
-    
-    if( 0 != numJobs ){
-      runner.setNumJobs(numJobs);
-    }
-    
-    try{
-      runner.run(); 
-    }catch(IOException e){
-      e.printStackTrace();
-    }
-  }
-  
-}

+ 308 - 0
src/test/org/apache/hadoop/mapred/MRBench.java

@@ -0,0 +1,308 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Runs a job multiple times and takes average of all runs.
+ *
+ * @author Sanjay Dahiya
+ * @author Nigel Daley
+ */
+public class MRBench {
+  
+  private static final Log LOG = LogFactory.getLog(MRBench.class);
+  private static Path BASE_DIR =
+    new Path(System.getProperty("test.build.data","/benchmarks/MRBench"));
+  private static Path INPUT_DIR = new Path(BASE_DIR, "mr_input");
+  private static Path OUTPUT_DIR = new Path(BASE_DIR, "mr_output");
+  
+  public static enum Order {RANDOM, ASCENDING, DESCENDING}; 
+  
+  /**
+   * Takes input format as text lines, runs some processing on it and 
+   * writes out data as text again. 
+   */
+  public static class Map extends MapReduceBase implements Mapper {
+    public void map(WritableComparable key, Writable value,
+        OutputCollector output, Reporter reporter) throws IOException 
+    {
+      String line = value.toString();
+      output.collect(new UTF8(process(line)), new UTF8(""));		
+    }
+    public String process(String line) {
+      return line; 
+    }
+  }
+
+  /**
+   * Ignores the key and writes values to the output. 
+   */
+  public static class Reduce extends MapReduceBase implements Reducer {
+    public void reduce(WritableComparable key, Iterator values,
+        OutputCollector output, Reporter reporter) throws IOException 
+    {
+      while(values.hasNext()) {
+        output.collect(key, new UTF8(values.next().toString()));
+      }
+    }
+  }
+
+  /**
+   * Generate a text file on the given filesystem with the given path name.
+   * The text file will contain the given number of lines of generated data.
+   * The generated data are string representations of numbers.  Each line
+   * is the same length, which is achieved by padding each number with
+   * an appropriate number of leading '0' (zero) characters.  The order of
+   * generated data is one of ascending, descending, or random.
+   */
+  public static void generateTextFile(FileSystem fs, Path inputFile, 
+    long numLines, Order sortOrder) throws IOException 
+  {
+    LOG.info("creating control file: "+numLines+" numLines, "+sortOrder+" sortOrder");
+    PrintStream output = null;
+    try {
+      output = new PrintStream(fs.create(inputFile));
+      int padding = String.valueOf(numLines).length();
+      switch(sortOrder) {
+        case RANDOM:
+          for (long l = 0; l < numLines; l++) {
+            output.println(pad((new Random()).nextLong(), padding));
+          }
+          break; 
+        case ASCENDING: 
+          for (long l = 0; l < numLines; l++) {
+            output.println(pad(l, padding));
+          }
+          break;
+        case DESCENDING: 
+          for (long l = numLines; l > 0; l--) {
+            output.println(pad(l, padding));
+          }
+          break;
+      }
+    } finally {
+      if (output != null)
+        output.close();
+    }
+    LOG.info("created control file: " + inputFile);
+  }
+  
+  /**
+   * Convert the given number to a string and pad the number with 
+   * leading '0' (zero) characters so that the string is exactly
+   * the given length.
+   */
+  private static String pad(long number, int length) {
+    String str = String.valueOf(number);
+    StringBuffer value = new StringBuffer(); 
+    for (int i = str.length(); i < length; i++) {
+      value.append("0"); 
+    }
+    value.append(str); 
+    return value.toString();
+  }
+  
+  /**
+   * Create the job configuration.
+   */
+  private static JobConf setupJob(int numMaps, int numReduces, String jarFile) {
+    JobConf jobConf = new JobConf(MRBench.class);
+    jobConf.addInputPath(INPUT_DIR);
+    
+    jobConf.setInputFormat(TextInputFormat.class);
+    jobConf.setOutputFormat(TextOutputFormat.class);
+    
+    jobConf.setInputKeyClass(LongWritable.class);
+    jobConf.setOutputValueClass(UTF8.class);
+    
+    jobConf.setMapOutputKeyClass(UTF8.class);
+    jobConf.setMapOutputValueClass(UTF8.class);
+    
+    if ( null != jarFile ) {
+      jobConf.setJar(jarFile);
+    }
+    jobConf.setMapperClass(Map.class);
+    jobConf.setReducerClass(Reduce.class);
+    
+    jobConf.setNumMapTasks(numMaps);
+    jobConf.setNumReduceTasks(numReduces);
+    
+    return jobConf; 
+  }
+  
+  /**
+   * Runs a MapReduce task, given number of times. The input to each run
+   * is the same file.
+   */
+  private static ArrayList<Long> runJobInSequence(JobConf masterJobConf, int numRuns) throws IOException {
+    Path intrimData = null; 
+    Random rand = new Random();
+    ArrayList<Long> execTimes = new ArrayList<Long>(); 
+    
+    for (int i = 0; i < numRuns; i++) {
+      // create a new job conf every time, reusing same object does not work 
+      JobConf jobConf = new JobConf(masterJobConf);
+      // reset the job jar because the copy constructor doesn't
+      jobConf.setJar(masterJobConf.getJar());
+      // give a new random name to output of the mapred tasks
+      jobConf.setOutputPath(new Path(OUTPUT_DIR, "output_" + rand.nextInt()));
+
+      LOG.info("Running job " + i + ":" +
+        " input=" + jobConf.getInputPaths()[0] + 
+        " output=" + jobConf.getOutputPath());
+      
+      // run the mapred task now 
+      long curTime = System.currentTimeMillis();
+      JobClient.runJob(jobConf);
+      execTimes.add(new Long(System.currentTimeMillis() - curTime));
+    }
+    return execTimes;
+  }
+  
+  /**
+   * <pre>
+   * Usage: mrbench
+   *    [-baseDir <base DFS path for output/input, default is /benchmarks/MRBench>]
+   *    [-jar <local path to job jar file containing Mapper and Reducer implementations, default is current jar file>]
+   *    [-numRuns <number of times to run the job, default is 1>]
+   *    [-maps <number of maps for each run, default is 2>]
+   *    [-reduces <number of reduces for each run, default is 1>]
+   *    [-inputLines <number of input lines to generate, default is 1>]
+   *    [-inputType <type of input to generate, one of ascending (default), descending, random>]
+   *    [-verbose]
+   * </pre>
+   */
+  public static void main (String[] args) throws IOException {
+    String version = "MRBenchmark.0.0.2";
+    System.out.println(version);
+
+    String usage = 
+      "Usage: mrbench " +
+      "[-baseDir <base DFS path for output/input, default is /benchmarks/MRBench>] " + 
+      "[-jar <local path to job jar file containing Mapper and Reducer implementations, default is current jar file>] " + 
+      "[-numRuns <number of times to run the job, default is 1>] " +
+      "[-maps <number of maps for each run, default is 2>] " +
+      "[-reduces <number of reduces for each run, default is 1>] " +
+      "[-inputLines <number of input lines to generate, default is 1>] " +
+      "[-inputType <type of input to generate, one of ascending (default), descending, random>] " + 
+      "[-verbose]";
+    
+    String jarFile = null;
+    int inputLines = 1; 
+    int numRuns = 1;
+    int numMaps = 2; 
+    int numReduces = 1;
+    boolean verbose = false;         
+    Order inputSortOrder = Order.ASCENDING;     
+    for (int i = 0; i < args.length; i++) { // parse command line
+      if (args[i].equals("-jar")) {
+        jarFile = args[++i];
+      } else if (args[i].equals("-numRuns")) {
+        numRuns = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-baseDir")) {
+        BASE_DIR = new Path(args[++i]);
+      } else if (args[i].equals("-maps")) {
+        numMaps = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-reduces")) {
+        numReduces = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-inputLines")) {
+        inputLines = Integer.parseInt(args[++i]);
+      } else if (args[i].equals("-inputType")) {
+        String s = args[++i]; 
+        if (s.equalsIgnoreCase("ascending")) {
+          inputSortOrder = Order.ASCENDING;
+        } else if (s.equalsIgnoreCase("descending")) {
+          inputSortOrder = Order.DESCENDING; 
+        } else if (s.equalsIgnoreCase("random")) {
+          inputSortOrder = Order.RANDOM;
+        } else {
+          inputSortOrder = null;
+        }
+      } else if (args[i].equals("-verbose")) {
+        verbose = true;
+      } else {
+        System.err.println(usage);
+        System.exit(-1);
+      }
+    }
+    
+    if (numRuns < 1 ||  // verify args
+      numMaps < 1 ||
+      numReduces < 1 ||
+      inputLines < 0 ||
+      inputSortOrder == null)
+    {
+      System.err.println(usage);
+      System.exit(-1);
+    }
+
+    JobConf jobConf = setupJob(numMaps, numReduces, jarFile);
+    FileSystem fs = FileSystem.get(jobConf);
+    Path inputFile = new Path(INPUT_DIR, "input_" + (new Random()).nextInt() + ".txt");
+    generateTextFile(fs, inputFile, inputLines, inputSortOrder);
+
+    // setup test output directory
+    fs.mkdirs(BASE_DIR); 
+    ArrayList<Long> execTimes = new ArrayList<Long>();
+    try {
+      execTimes = runJobInSequence(jobConf, numRuns);
+    } finally {
+      // delete output -- should we really do this?
+      fs.delete(BASE_DIR);
+    }
+    
+    if (verbose) {
+      // Print out a report 
+      System.out.println("Total MapReduce jobs executed: " + numRuns);
+      System.out.println("Total lines of data per job: " + inputLines);
+      System.out.println("Maps per job: " + numMaps);
+      System.out.println("Reduces per job: " + numReduces);
+    }
+    int i = 0;
+    long totalTime = 0; 
+    for (Long time : execTimes) {
+      totalTime += time.longValue(); 
+      if (verbose) {
+        System.out.println("Total milliseconds for task: " + (++i) + 
+            " = " +  time);
+      }
+    }
+    long avgTime = totalTime / numRuns;    
+    System.out.println("DataLines\tMaps\tReduces\tAvgTime (milliseconds)");
+    System.out.println(inputLines + "\t\t" + numMaps + "\t" + 
+        numReduces + "\t" + avgTime);
+  }
+  
+}

+ 24 - 24
src/test/org/apache/hadoop/test/AllTestDriver.java

@@ -19,49 +19,49 @@
 package org.apache.hadoop.test;
 
 import org.apache.hadoop.util.ProgramDriver;
+import org.apache.hadoop.mapred.MRBench;
 import org.apache.hadoop.mapred.TestMapRed;
 import org.apache.hadoop.mapred.TestTextInputFormat;
 import org.apache.hadoop.mapred.TestSequenceFileInputFormat;
 import org.apache.hadoop.dfs.ClusterTestDFS;
 import org.apache.hadoop.dfs.NNBench;
+import org.apache.hadoop.fs.DistributedFSCheck;
+import org.apache.hadoop.fs.TestDFSIO;
+import org.apache.hadoop.fs.DFSCIOTest;
 import org.apache.hadoop.fs.TestFileSystem;
 import org.apache.hadoop.io.TestArrayFile;
 import org.apache.hadoop.io.TestSetFile;
 import org.apache.hadoop.io.TestSequenceFile;
 import org.apache.hadoop.ipc.TestIPC;
 import org.apache.hadoop.ipc.TestRPC;
-import org.apache.hadoop.fs.DistributedFSCheck;
-import org.apache.hadoop.fs.TestDFSIO;
-import org.apache.hadoop.fs.DFSCIOTest;
 
 public class AllTestDriver {
   
   /**
    * A description of the test program for running all the tests using jar file
-   * @date April 2006
    */
-    
-    public static void main(String argv[]){
-	ProgramDriver pgd = new ProgramDriver();
-	try {
-            pgd.addClass("nnbench", NNBench.class, "A benchmark that stresses the namenode.");
-	    pgd.addClass("mapredtest", TestMapRed.class, "A map/reduce test check.");
-	    pgd.addClass("clustertestdfs", ClusterTestDFS.class, "A pseudo distributed test for DFS.");
-	    pgd.addClass("testfilesystem", TestFileSystem.class, "A test for FileSystem read/write.");
-	    pgd.addClass("testsequencefile", TestSequenceFile.class, "A test for flat files of binary key value pairs.");
-	    pgd.addClass("testsetfile", TestSetFile.class, "A test for flat files of binary key/value pairs.");
-	    pgd.addClass("testarrayfile", TestArrayFile.class, "A test for flat files of binary key/value pairs.");
-	    pgd.addClass("testrpc", TestRPC.class, "A test for rpc.");
-	    pgd.addClass("testipc", TestIPC.class, "A test for ipc.");
-	    pgd.addClass("testsequencefileinputformat", TestSequenceFileInputFormat.class, "A test for sequence file input format.");
-	    pgd.addClass("testtextinputformat", TestTextInputFormat.class, "A test for text input format.");
+  public static void main(String argv[]){
+    ProgramDriver pgd = new ProgramDriver();
+    try {
+      pgd.addClass("mrbench", MRBench.class, "A map/reduce benchmark that can create many small jobs");
+      pgd.addClass("nnbench", NNBench.class, "A benchmark that stresses the namenode.");
+      pgd.addClass("mapredtest", TestMapRed.class, "A map/reduce test check.");
+      pgd.addClass("clustertestdfs", ClusterTestDFS.class, "A pseudo distributed test for DFS.");
+      pgd.addClass("testfilesystem", TestFileSystem.class, "A test for FileSystem read/write.");
+      pgd.addClass("testsequencefile", TestSequenceFile.class, "A test for flat files of binary key value pairs.");
+      pgd.addClass("testsetfile", TestSetFile.class, "A test for flat files of binary key/value pairs.");
+      pgd.addClass("testarrayfile", TestArrayFile.class, "A test for flat files of binary key/value pairs.");
+      pgd.addClass("testrpc", TestRPC.class, "A test for rpc.");
+      pgd.addClass("testipc", TestIPC.class, "A test for ipc.");
+      pgd.addClass("testsequencefileinputformat", TestSequenceFileInputFormat.class, "A test for sequence file input format.");
+      pgd.addClass("testtextinputformat", TestTextInputFormat.class, "A test for text input format.");
       pgd.addClass("TestDFSIO", TestDFSIO.class, "Distributed i/o benchmark.");
       pgd.addClass("DFSCIOTest", DFSCIOTest.class, "Distributed i/o benchmark of libhdfs.");
       pgd.addClass("DistributedFSCheck", DistributedFSCheck.class, "Distributed checkup of the file system consistency.");
-	    pgd.driver(argv);
-	}
-	catch(Throwable e){
-	    e.printStackTrace();
-	}
+      pgd.driver(argv);
+    } catch(Throwable e) {
+      e.printStackTrace();
     }
+  }
 }
+