Browse Source

HDFS-641. Move all of the components that depend on map/reduce to
map/reduce. (omalley)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@817853 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 15 years ago
parent
commit
ff85e557fc
28 changed files with 4 additions and 6596 deletions
  1. 3 0
      CHANGES.txt
  2. 1 58
      build.xml
  3. BIN
      lib/hadoop-mapred-0.22.0-dev.jar
  4. BIN
      lib/hadoop-mapred-examples-0.22.0-dev.jar
  5. BIN
      lib/hadoop-mapred-test-0.22.0-dev.jar
  6. 0 25
      src/contrib/block_forensics/README
  7. 0 66
      src/contrib/block_forensics/build.xml
  8. 0 186
      src/contrib/block_forensics/client/BlockForensics.java
  9. 0 44
      src/contrib/block_forensics/ivy.xml
  10. 0 21
      src/contrib/block_forensics/ivy/libraries.properties
  11. 0 136
      src/contrib/block_forensics/src/java/org/apache/hadoop/block_forensics/BlockSearch.java
  12. 0 101
      src/test/hdfs-with-mr/org/apache/hadoop/fs/AccumulatingReducer.java
  13. 0 550
      src/test/hdfs-with-mr/org/apache/hadoop/fs/DFSCIOTest.java
  14. 0 357
      src/test/hdfs-with-mr/org/apache/hadoop/fs/DistributedFSCheck.java
  15. 0 125
      src/test/hdfs-with-mr/org/apache/hadoop/fs/IOMapperBase.java
  16. 0 1129
      src/test/hdfs-with-mr/org/apache/hadoop/fs/JHLogAnalyzer.java
  17. 0 456
      src/test/hdfs-with-mr/org/apache/hadoop/fs/TestDFSIO.java
  18. 0 675
      src/test/hdfs-with-mr/org/apache/hadoop/fs/TestFileSystem.java
  19. 0 146
      src/test/hdfs-with-mr/org/apache/hadoop/fs/TestJHLA.java
  20. 0 975
      src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBench.java
  21. 0 359
      src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
  22. 0 603
      src/test/hdfs-with-mr/org/apache/hadoop/io/FileBench.java
  23. 0 98
      src/test/hdfs-with-mr/org/apache/hadoop/io/TestSequenceFileMergeProgress.java
  24. 0 197
      src/test/hdfs-with-mr/org/apache/hadoop/ipc/TestSocketFactory.java
  25. 0 152
      src/test/hdfs-with-mr/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java
  26. 0 46
      src/test/hdfs-with-mr/org/apache/hadoop/test/AllTestDriver.java
  27. 0 78
      src/test/hdfs-with-mr/org/apache/hadoop/test/HdfsWithMRTestDriver.java
  28. 0 13
      src/test/mapred-site.xml

+ 3 - 0
CHANGES.txt

@@ -187,6 +187,9 @@ Release 0.21.0 - Unreleased
 
     HDFS-598. Eclipse launch task for HDFS. (Eli Collins via tomwhite)
 
+    HDFS-641. Move all of the components that depend on map/reduce to 
+    map/reduce. (omalley)
+
   BUG FIXES
 
     HDFS-76. Better error message to users when commands fail because of 

+ 1 - 58
build.xml

@@ -194,15 +194,6 @@
     <path refid="classpath"/>
   </path>
 
-  <path id="test.hdfs.with.mr.classpath">
-    <path refid="test.classpath"/>
-    <pathelement location="${test.hdfs.with.mr.build.classes}" />
-    <pathelement location="${lib.dir}/hadoop-mapred-test-${hadoop-mr.version}.jar" />
-    <pathelement location="${lib.dir}/hadoop-mapred-${hadoop-mr.version}.jar" />
-    <pathelement location="${lib.dir}/hadoop-mapred-tools-${hadoop-mr.version}.jar" />
-    <pathelement location="${lib.dir}/hadoop-mapred-examples-${hadoop-mr.version}.jar" />
-  </path>
-
   <!-- the cluster test classpath: uses conf.dir for configuration -->
   <path id="test.cluster.classpath">
     <path refid="classpath"/>
@@ -253,12 +244,6 @@
         <exclude name="**/*.jsp" />
       </fileset>
     </copy>
-    <unzip src="${lib.dir}/hadoop-mapred-${hadoop-mr.version}.jar"
-        dest="${build.dir}">
-      <patternset>
-        <include name="webapps/**"/>
-      </patternset>
-    </unzip>
 
     <copy todir="${conf.dir}" verbose="true">
       <fileset dir="${conf.dir}" includes="**/*.template"/>
@@ -656,48 +641,6 @@
      <macro-test-runner test.file="${test.hdfs.commit.tests.file}" />
   </target>
 
-  <target name="run-test-hdfs-with-mr" depends="compile-hdfs-with-mr-test" description="Run hdfs unit tests that require mapred">
-
-    <delete dir="${test.build.data}"/>
-    <mkdir dir="${test.build.data}"/>
-    <delete dir="${test.log.dir}"/>
-    <mkdir dir="${test.log.dir}"/>
-    <copy file="${test.src.dir}/hadoop-policy.xml" 
-      todir="${test.build.extraconf}" />
-    <copy file="${test.src.dir}/fi-site.xml"
-      todir="${test.build.extraconf}" />
-    <junit showoutput="${test.output}"
-      printsummary="${test.junit.printsummary}"
-      haltonfailure="${test.junit.haltonfailure}"
-      fork="yes"
-      forkmode="${test.junit.fork.mode}"
-      maxmemory="${test.junit.maxmemory}"
-      dir="${basedir}" timeout="${test.timeout}"
-      errorProperty="tests.failed" failureProperty="tests.failed">
-      <sysproperty key="test.build.data" value="${test.build.data}"/>
-      <sysproperty key="test.cache.data" value="${test.cache.data}"/>     
-      <sysproperty key="test.debug.data" value="${test.debug.data}"/>
-      <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
-      <sysproperty key="test.src.dir" value="${test.src.dir}"/>
-      <sysproperty key="test.build.extraconf" value="${test.build.extraconf}" />
-      <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml"/>
-      <classpath refid="test.hdfs.with.mr.classpath"/>
-      <syspropertyset id="FaultProbabilityProperties">
-        <propertyref regex="fi.*"/>
-      </syspropertyset>
-      <formatter type="${test.junit.output.format}" />
-      <batchtest todir="${test.build.dir}" if="tests.notestcase">
-        <fileset dir="${test.src.dir}/hdfs-with-mr"
-           includes="**/${test.include}.java"
-           excludes="**/${test.exclude}.java" />
-      </batchtest>
-      <batchtest todir="${test.build.dir}" if="tests.testcase">
-        <fileset dir="${test.src.dir}/hdfs-with-mr" includes="**/${testcase}.java"/>
-      </batchtest>
-    </junit>
-    <antcall target="checkfailure"/>
-  </target>  
-
   <target name="checkfailure" if="tests.failed">
     <touch file="${test.build.dir}/testsfailed"/>
     <fail unless="continueOnFailure">Tests failed!</fail>
@@ -712,7 +655,7 @@
     </subant> 
   </target>
 
-  <target name="test-core" description="Run core, hdfs and mapred unit tests">
+  <target name="test-core" description="Run hdfs unit tests">
     <delete file="${test.build.dir}/testsfailed"/>
     <property name="continueOnFailure" value="true"/>
     <antcall target="run-test-hdfs"/>

BIN
lib/hadoop-mapred-0.22.0-dev.jar


BIN
lib/hadoop-mapred-examples-0.22.0-dev.jar


BIN
lib/hadoop-mapred-test-0.22.0-dev.jar


+ 0 - 25
src/contrib/block_forensics/README

@@ -1,25 +0,0 @@
-This contribution consists of two components designed to make it easier to find information about lost or corrupt blocks.
-
-The first is a map reduce designed to search for one or more block ids in a set of log files. It exists in org.apache.hadoop.block_forensics.BlockSearch. Building this contribution generates a jar file that can be executed using:
-
-  bin/hadoop jar [jar location] [hdfs input path] [hdfs output dir] [comma delimited list of block ids]
-
-  For example, the command:
-    bin/hadoop jar /foo/bar/hadoop-0.1-block_forensics.jar /input/* /ouptut 2343,45245,75823
-  ... searches for any of blocks 2343, 45245, or 75823 in any of the files 
-   contained in the /input/ directory. 
-
-  
-  The output will be any line containing one of the provided block ids. While this tool is designed to be used with block ids, it can also be used for general text searching. 
-  
-The second component is a standalone java program that will repeatedly query the namenode at a given interval looking for corrupt replicas. If it finds a corrupt replica, it will launch the above map reduce job. The syntax is:
-
-  java BlockForensics http://[namenode]:[port]/corrupt_replicas_xml.jsp [sleep time between namenode query for corrupt blocks (in milliseconds)] [mapred jar location] [hdfs input path]
-
-  For example, the command:
-    java BlockForensics http://localhost:50070/corrupt_replicas_xml.jsp 30000
-                        /foo/bar/hadoop-0.1-block_forensics.jar /input/*
-  ... queries the namenode at localhost:50070 for corrupt replicas every 30
-      seconds and runs /foo/bar/hadoop-0.1-block_forensics.jar if any are found. 
-
-The map reduce job jar and the BlockForensics class can be found in your build/contrib/block_forensics and build/contrib/block_forensics/classes directories, respectively. 

+ 0 - 66
src/contrib/block_forensics/build.xml

@@ -1,66 +0,0 @@
-<?xml version="1.0"?>
-
-<!--
-   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.
--->
-
-<!-- 
-Before you can run these subtargets directly, you need 
-to call at top-level: ant deploy-contrib compile-core-test
--->
-<project name="block_forensics" default="jar">
-  <property name="version" value="0.1"/>
-  <import file="../build-contrib.xml"/>
-
-  <!-- create the list of files to add to the classpath -->
-  <fileset dir="${hadoop.root}/lib" id="class.path">
-    <include name="**/*.jar" />
-    <exclude name="**/excluded/" />
-  </fileset>
-
-  <!-- Override jar target to specify main class -->
-  <target name="jar" depends="compile">
-    <jar
-      jarfile="${build.dir}/hadoop-${version}-${name}.jar"
-      basedir="${build.classes}"      
-    >
-    <manifest>
-      <attribute name="Main-Class" value="org.apache.hadoop.blockforensics.BlockSearch"/>
-    </manifest>
-    </jar>
-    
-    <javac srcdir="client" destdir="${build.classes}"/>
-
-  </target>
-
-  <!-- Run only pure-Java unit tests. superdottest -->
-  <target name="test">
-   <antcall target="hadoopbuildcontrib.test"> 
-   </antcall>
-  </target>  
- 
-  <!-- Run all unit tests
-  This is not called as part of the nightly build
-  because it will only run on platforms that have standard 
-  Unix utilities available. 
-  -->
- <target name="test-unix">
-   <antcall target="hadoopbuildcontrib.test">
-   </antcall>
- </target>  
-
-
-</project>

+ 0 - 186
src/contrib/block_forensics/client/BlockForensics.java

@@ -1,186 +0,0 @@
-/**
- * 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.
- */
- 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.lang.Runtime;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.TreeSet;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import org.w3c.dom.Document;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-/**
- * This class repeatedly queries a namenode looking for corrupt replicas. If 
- * any are found a provided hadoop job is launched and the output printed
- * to stdout. 
- *
- * The syntax is:
- *
- * java BlockForensics http://[namenode]:[port]/corrupt_replicas_xml.jsp 
- *                    [sleep time between namenode query for corrupt blocks
- *                      (in seconds)] [mapred jar location] [hdfs input path]
- *
- * All arguments are required.
- */
-public class BlockForensics {
-  
-  public static String join(List<?> l, String sep) {
-    StringBuilder sb = new StringBuilder();
-    Iterator it = l.iterator();
-    
-    while(it.hasNext()){
-      sb.append(it.next());
-      if (it.hasNext()) {
-        sb.append(sep);
-      }
-    }
-    
-    return sb.toString();
-  }
-  
-  
-  // runs hadoop command and prints output to stdout
-  public static void runHadoopCmd(String ... args)
-  throws IOException {
-    String hadoop_home = System.getenv("HADOOP_HOME");
-    
-    List<String> l = new LinkedList<String>();
-    l.add("bin/hadoop");
-    l.addAll(Arrays.asList(args));
-    
-    ProcessBuilder pb = new ProcessBuilder(l);
-    
-    if (hadoop_home != null) {
-      pb.directory(new File(hadoop_home));
-    }
-
-    pb.redirectErrorStream(true);
-          
-    Process p = pb.start();
-
-    BufferedReader br = new BufferedReader(
-                          new InputStreamReader(p.getInputStream()));
-    String line;
-
-    while ((line = br.readLine()) != null) {
-      System.out.println(line);
-    }
-
-
-  }
-    
-  public static void main(String[] args)
-    throws SAXException, ParserConfigurationException, 
-           InterruptedException, IOException {
-
-    if (System.getenv("HADOOP_HOME") == null) {
-      System.err.println("The environmental variable HADOOP_HOME is undefined");
-      System.exit(1);
-    }
-
-
-    if (args.length < 4) {
-      System.out.println("Usage: java BlockForensics [http://namenode:port/"
-                         + "corrupt_replicas_xml.jsp] [sleep time between "
-                         + "requests (in milliseconds)] [mapred jar location] "
-                         + "[hdfs input path]");
-      return;
-    }
-             
-    int sleepTime = 30000;
-  
-    try {
-      sleepTime = Integer.parseInt(args[1]);
-    } catch (NumberFormatException e) {
-      System.out.println("The sleep time entered is invalid, "
-                         + "using default value: "+sleepTime+"ms"); 
-    }
-      
-    Set<Long> blockIds = new TreeSet<Long>();
-    
-    while (true) {
-      InputStream xml = new URL(args[0]).openConnection().getInputStream();
-    
-      DocumentBuilderFactory fact = DocumentBuilderFactory.newInstance();
-      DocumentBuilder builder = fact.newDocumentBuilder();
-      Document doc = builder.parse(xml);
-         
-      NodeList corruptReplicaNodes = doc.getElementsByTagName("block_id");
-
-      List<Long> searchBlockIds = new LinkedList<Long>();
-      for(int i=0; i<corruptReplicaNodes.getLength(); i++) {
-        Long blockId = new Long(corruptReplicaNodes.item(i)
-                                                    .getFirstChild()
-                                                    .getNodeValue());
-        if (!blockIds.contains(blockId)) {
-          blockIds.add(blockId);
-          searchBlockIds.add(blockId);
-        }
-      }
-      
-      if (searchBlockIds.size() > 0) {
-        String blockIdsStr = BlockForensics.join(searchBlockIds, ",");
-        System.out.println("\nSearching for: " + blockIdsStr);
-        String tmpDir =
-            new String("/tmp-block-forensics-" +
-                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
-
-        System.out.println("Using temporary dir: "+tmpDir);
-
-        // delete tmp dir
-        BlockForensics.runHadoopCmd("fs", "-rmr", tmpDir);
-      
-        // launch mapred job      
-        BlockForensics.runHadoopCmd("jar",
-                                    args[2], // jar location
-                                    args[3], // input dir
-                                    tmpDir, // output dir
-                                    blockIdsStr// comma delimited list of blocks
-                                    );
-        // cat output
-        BlockForensics.runHadoopCmd("fs", "-cat", tmpDir+"/part*");
-
-        // delete temp dir
-        BlockForensics.runHadoopCmd("fs", "-rmr", tmpDir);
-
-        int sleepSecs = (int)(sleepTime/1000.);
-        System.out.print("Sleeping for "+sleepSecs
-                         + " second"+(sleepSecs == 1?"":"s")+".");
-      }
-
-      System.out.print(".");
-      Thread.sleep(sleepTime);
-
-    }
-  }
-}

+ 0 - 44
src/contrib/block_forensics/ivy.xml

@@ -1,44 +0,0 @@
-<?xml version="1.0" ?>
-
-<!--
-   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.
--->
-
-<ivy-module version="1.0">
-  <info organisation="org.apache.hadoop" module="${ant.project.name}">
-    <license name="Apache 2.0"/>
-    <ivyauthor name="Apache Hadoop Team" url="http://hadoop.apache.org"/>
-    <description>
-        Apache Hadoop
-    </description>
-  </info>
-  <configurations defaultconfmapping="default">
-    <!--these match the Maven configurations-->
-    <conf name="default" extends="master,runtime"/>
-    <conf name="master" description="contains the artifact but no dependencies"/>
-    <conf name="runtime" description="runtime but not the artifact" />
-
-    <conf name="common" visibility="private" 
-      extends="runtime"
-      description="artifacts needed to compile/test the application"/>
-    <conf name="test" visibility="private" extends="runtime"/>
-  </configurations>
-
-  <publications>
-    <!--get the artifact from our module name-->
-    <artifact conf="master"/>
-  </publications>
-</ivy-module>

+ 0 - 21
src/contrib/block_forensics/ivy/libraries.properties

@@ -1,21 +0,0 @@
-# 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.
-
-
-#This properties file lists the versions of the various artifacts used by thrifts.
-#It drives ivy and the generation of a maven POM
-
-#Please list the dependencies name with version if they are different from the ones 
-#listed in the global libraries.properties file (in alphabetical order)

+ 0 - 136
src/contrib/block_forensics/src/java/org/apache/hadoop/block_forensics/BlockSearch.java

@@ -1,136 +0,0 @@
-/**
- * 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.blockforensics;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.StringTokenizer;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-
-/**
- * BlockSearch is a mapred job that's designed to search input for appearances 
- * of strings. 
- *
- * The syntax is:
- * 
- * bin/hadoop jar [jar location] [hdfs input path] [hdfs output dir]
-                  [comma delimited list of block ids]
- *
- * All arguments are required.
- *
- * This tool is designed to be used to search for one or more block ids in log
- * files but can be used for general text search, assuming the search strings
- * don't contain tokens. It assumes only one search string will appear per line. 
- */
-public class BlockSearch extends Configured implements Tool {
-  public static class Map extends Mapper<LongWritable, Text, Text, Text> {
-    private Text blockIdText = new Text();
-    private Text valText = new Text();
-    private List<String> blockIds = null;
-
-    protected void setup(Context context) 
-      throws IOException, InterruptedException {
-      Configuration conf = context.getConfiguration();
-      StringTokenizer st = new StringTokenizer(conf.get("blockIds"), ",");
-      blockIds = new LinkedList<String>();
-      while (st.hasMoreTokens()) {
-        String blockId = st.nextToken();
-        blockIds.add(blockId);
-      }
-    }
-
-
-    public void map(LongWritable key, Text value, Context context)
-      throws IOException, InterruptedException {
-      if (blockIds == null) {
-        System.err.println("Error: No block ids specified");
-      } else {
-        String valStr = value.toString();
-
-        for(String blockId: blockIds) {
-          if (valStr.indexOf(blockId) != -1) {
-            blockIdText.set(blockId);
-            valText.set(valStr);
-            context.write(blockIdText, valText);
-            break; // assume only one block id appears per line
-          }
-        }
-      }
-
-    }
-
-  }
-
-
-  public static class Reduce extends Reducer<Text, Text, Text, Text> {
-    private Text val = new Text();
-    public void reduce(Text key, Iterator<Text> values, Context context)
-    throws IOException, InterruptedException {
-      while (values.hasNext()) {
-        context.write(key, values.next());
-      }
-    }
-  }
-    
-  public int run(String[] args) throws Exception {
-    if (args.length < 3) {
-      System.out.println("BlockSearch <inLogs> <outDir> <comma delimited list of blocks>");
-      ToolRunner.printGenericCommandUsage(System.out);
-      return 2;
-    }
-
-    Configuration conf = getConf();
-    conf.set("blockIds", args[2]);
-
-    Job job = new Job(conf);
-
-    job.setCombinerClass(Reduce.class);
-    job.setJarByClass(BlockSearch.class);
-    job.setJobName("BlockSearch");
-    job.setMapperClass(Map.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setReducerClass(Reduce.class);
-
-    FileInputFormat.setInputPaths(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new BlockSearch(), args);
-    System.exit(res);
-  }
-}

+ 0 - 101
src/test/hdfs-with-mr/org/apache/hadoop/fs/AccumulatingReducer.java

@@ -1,101 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
-
-/**
- * Reducer that accumulates values based on their type.
- * <p>
- * The type is specified in the key part of the key-value pair 
- * as a prefix to the key in the following way
- * <p>
- * <tt>type:key</tt>
- * <p>
- * The values are accumulated according to the types:
- * <ul>
- * <li><tt>s:</tt> - string, concatenate</li>
- * <li><tt>f:</tt> - float, summ</li>
- * <li><tt>l:</tt> - long, summ</li>
- * </ul>
- * 
- */
-@SuppressWarnings("deprecation")
-public class AccumulatingReducer extends MapReduceBase
-    implements Reducer<Text, Text, Text, Text> {
-  static final String VALUE_TYPE_LONG = "l:";
-  static final String VALUE_TYPE_FLOAT = "f:";
-  static final String VALUE_TYPE_STRING = "s:";
-  private static final Log LOG = LogFactory.getLog(AccumulatingReducer.class);
-  
-  protected String hostName;
-  
-  public AccumulatingReducer () {
-    LOG.info("Starting AccumulatingReducer !!!");
-    try {
-      hostName = java.net.InetAddress.getLocalHost().getHostName();
-    } catch(Exception e) {
-      hostName = "localhost";
-    }
-    LOG.info("Starting AccumulatingReducer on " + hostName);
-  }
-  
-  public void reduce(Text key, 
-                     Iterator<Text> values,
-                     OutputCollector<Text, Text> output, 
-                     Reporter reporter
-                     ) throws IOException {
-    String field = key.toString();
-
-    reporter.setStatus("starting " + field + " ::host = " + hostName);
-
-    // concatenate strings
-    if (field.startsWith(VALUE_TYPE_STRING)) {
-      StringBuffer sSum = new StringBuffer();
-      while (values.hasNext())
-        sSum.append(values.next().toString()).append(";");
-      output.collect(key, new Text(sSum.toString()));
-      reporter.setStatus("finished " + field + " ::host = " + hostName);
-      return;
-    }
-    // sum long values
-    if (field.startsWith(VALUE_TYPE_FLOAT)) {
-      float fSum = 0;
-      while (values.hasNext())
-        fSum += Float.parseFloat(values.next().toString());
-      output.collect(key, new Text(String.valueOf(fSum)));
-      reporter.setStatus("finished " + field + " ::host = " + hostName);
-      return;
-    }
-    // sum long values
-    if (field.startsWith(VALUE_TYPE_LONG)) {
-      long lSum = 0;
-      while (values.hasNext()) {
-        lSum += Long.parseLong(values.next().toString());
-      }
-      output.collect(key, new Text(String.valueOf(lSum)));
-    }
-    reporter.setStatus("finished " + field + " ::host = " + hostName);
-  }
-}

+ 0 - 550
src/test/hdfs-with-mr/org/apache/hadoop/fs/DFSCIOTest.java

@@ -1,550 +0,0 @@
- /**
- * 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.fs;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
-import java.util.Date;
-import java.util.StringTokenizer;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.mapred.*;
-
-/**
- * Distributed i/o benchmark.
- * <p>
- * This test writes into or reads from a specified number of files.
- * File size is specified as a parameter to the test. 
- * Each file is accessed in a separate map task.
- * <p>
- * The reducer collects the following statistics:
- * <ul>
- * <li>number of tasks completed</li>
- * <li>number of bytes written/read</li>
- * <li>execution time</li>
- * <li>io rate</li>
- * <li>io rate squared</li>
- * </ul>
- *    
- * Finally, the following information is appended to a local file
- * <ul>
- * <li>read or write test</li>
- * <li>date and time the test finished</li>   
- * <li>number of files</li>
- * <li>total number of bytes processed</li>
- * <li>throughput in mb/sec (total number of bytes / sum of processing times)</li>
- * <li>average i/o rate in mb/sec per file</li>
- * <li>standard i/o rate deviation</li>
- * </ul>
- */
-public class DFSCIOTest extends TestCase {
-  // Constants
-  private static final Log LOG = LogFactory.getLog(DFSCIOTest.class);
-  private static final int TEST_TYPE_READ = 0;
-  private static final int TEST_TYPE_WRITE = 1;
-  private static final int TEST_TYPE_CLEANUP = 2;
-  private static final int DEFAULT_BUFFER_SIZE = 1000000;
-  private static final String BASE_FILE_NAME = "test_io_";
-  private static final String DEFAULT_RES_FILE_NAME = "DFSCIOTest_results.log";
-  
-  private static Configuration fsConfig = new Configuration();
-  private static final long MEGA = 0x100000;
-  private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/DFSCIOTest");
-  private static Path CONTROL_DIR = new Path(TEST_ROOT_DIR, "io_control");
-  private static Path WRITE_DIR = new Path(TEST_ROOT_DIR, "io_write");
-  private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");
-  private static Path DATA_DIR = new Path(TEST_ROOT_DIR, "io_data");
-
-  private static Path HDFS_TEST_DIR = new Path("/tmp/DFSCIOTest");
-  private static String HDFS_LIB_VERSION = System.getProperty("libhdfs.version", "1");
-  private static String CHMOD = new String("chmod");
-  private static Path HDFS_SHLIB = new Path(HDFS_TEST_DIR + "/libhdfs.so." + HDFS_LIB_VERSION);
-  private static Path HDFS_READ = new Path(HDFS_TEST_DIR + "/hdfs_read");
-  private static Path HDFS_WRITE = new Path(HDFS_TEST_DIR + "/hdfs_write");
-
-  /**
-   * Run the test with default parameters.
-   * 
-   * @throws Exception
-   */
-  public void testIOs() throws Exception {
-    testIOs(10, 10);
-  }
-
-  /**
-   * Run the test with the specified parameters.
-   * 
-   * @param fileSize file size
-   * @param nrFiles number of files
-   * @throws IOException
-   */
-  public static void testIOs(int fileSize, int nrFiles)
-    throws IOException {
-
-    FileSystem fs = FileSystem.get(fsConfig);
-
-    createControlFile(fs, fileSize, nrFiles);
-    writeTest(fs);
-    readTest(fs);
-  }
-
-  private static void createControlFile(
-                                        FileSystem fs,
-                                        int fileSize, // in MB 
-                                        int nrFiles
-                                        ) throws IOException {
-    LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
-
-    fs.delete(CONTROL_DIR, true);
-
-    for(int i=0; i < nrFiles; i++) {
-      String name = getFileName(i);
-      Path controlFile = new Path(CONTROL_DIR, "in_file_" + name);
-      SequenceFile.Writer writer = null;
-      try {
-        writer = SequenceFile.createWriter(fs, fsConfig, controlFile,
-                                           Text.class, LongWritable.class,
-                                           CompressionType.NONE);
-        writer.append(new Text(name), new LongWritable(fileSize));
-      } catch(Exception e) {
-        throw new IOException(e.getLocalizedMessage());
-      } finally {
-    	if (writer != null)
-          writer.close();
-    	writer = null;
-      }
-    }
-    LOG.info("created control files for: "+nrFiles+" files");
-  }
-
-  private static String getFileName(int fIdx) {
-    return BASE_FILE_NAME + Integer.toString(fIdx);
-  }
-  
-  /**
-   * Write/Read mapper base class.
-   * <p>
-   * Collects the following statistics per task:
-   * <ul>
-   * <li>number of tasks completed</li>
-   * <li>number of bytes written/read</li>
-   * <li>execution time</li>
-   * <li>i/o rate</li>
-   * <li>i/o rate squared</li>
-   * </ul>
-   */
-  private abstract static class IOStatMapper extends IOMapperBase<Long> {
-    IOStatMapper() { 
-    }
-    
-    void collectStats(OutputCollector<Text, Text> output, 
-                      String name,
-                      long execTime, 
-                      Long objSize) throws IOException {
-      long totalSize = objSize.longValue();
-      float ioRateMbSec = (float)totalSize * 1000 / (execTime * MEGA);
-      LOG.info("Number of bytes processed = " + totalSize);
-      LOG.info("Exec time = " + execTime);
-      LOG.info("IO rate = " + ioRateMbSec);
-      
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "tasks"),
-          new Text(String.valueOf(1)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "size"),
-          new Text(String.valueOf(totalSize)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "time"),
-          new Text(String.valueOf(execTime)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_FLOAT + "rate"),
-          new Text(String.valueOf(ioRateMbSec*1000)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_FLOAT + "sqrate"),
-          new Text(String.valueOf(ioRateMbSec*ioRateMbSec*1000)));
-    }
-  }
-
-  /**
-   * Write mapper class.
-   */
-  public static class WriteMapper extends IOStatMapper {
-
-    public WriteMapper() { 
-      super(); 
-      for(int i=0; i < bufferSize; i++)
-        buffer[i] = (byte)('0' + i % 50);
-    }
-
-    public Long doIO(Reporter reporter, 
-                       String name, 
-                       long totalSize 
-                       ) throws IOException {
-      // create file
-      totalSize *= MEGA;
-      
-      // create instance of local filesystem 
-      FileSystem localFS = FileSystem.getLocal(fsConfig);
-      
-      try {
-        // native runtime
-        Runtime runTime = Runtime.getRuntime();
-          
-        // copy the dso and executable from dfs and chmod them
-        synchronized (this) {
-          localFS.delete(HDFS_TEST_DIR, true);
-          if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
-            throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
-          }
-        }
-        
-        synchronized (this) {
-          if (!localFS.exists(HDFS_SHLIB)) {
-            FileUtil.copy(fs, HDFS_SHLIB, localFS, HDFS_SHLIB, false, fsConfig);
-
-            String chmodCmd = new String(CHMOD + " a+x " + HDFS_SHLIB);
-            Process process = runTime.exec(chmodCmd);
-            int exitStatus = process.waitFor();
-            if (exitStatus != 0) {
-              throw new IOException(chmodCmd + ": Failed with exitStatus: " + exitStatus);
-            }
-          }
-        } 
-        
-        synchronized (this) {
-          if (!localFS.exists(HDFS_WRITE)) {
-            FileUtil.copy(fs, HDFS_WRITE, localFS, HDFS_WRITE, false, fsConfig);
-
-            String chmodCmd = new String(CHMOD + " a+x " + HDFS_WRITE); 
-            Process process = runTime.exec(chmodCmd);
-            int exitStatus = process.waitFor();
-            if (exitStatus != 0) {
-              throw new IOException(chmodCmd + ": Failed with exitStatus: " + exitStatus);
-            }
-          }
-        }
-    	  	  
-        // exec the C program
-        Path outFile = new Path(DATA_DIR, name);
-        String writeCmd = new String(HDFS_WRITE + " " + outFile + " " + totalSize + " " + bufferSize); 
-        Process process = runTime.exec(writeCmd, null, new File(HDFS_TEST_DIR.toString()));
-        int exitStatus = process.waitFor();
-        if (exitStatus != 0) {
-          throw new IOException(writeCmd + ": Failed with exitStatus: " + exitStatus);
-        }
-      } catch (InterruptedException interruptedException) {
-        reporter.setStatus(interruptedException.toString());
-      } finally {
-        localFS.close();
-      }
-      return new Long(totalSize);
-    }
-  }
-
-  private static void writeTest(FileSystem fs)
-    throws IOException {
-
-    fs.delete(DATA_DIR, true);
-    fs.delete(WRITE_DIR, true);
-    
-    runIOTest(WriteMapper.class, WRITE_DIR);
-  }
-  
-  private static void runIOTest( Class<? extends Mapper> mapperClass, 
-                                 Path outputDir
-                                 ) throws IOException {
-    JobConf job = new JobConf(fsConfig, DFSCIOTest.class);
-
-    FileInputFormat.setInputPaths(job, CONTROL_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(mapperClass);
-    job.setReducerClass(AccumulatingReducer.class);
-
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-  /**
-   * Read mapper class.
-   */
-  public static class ReadMapper extends IOStatMapper {
-
-    public ReadMapper() { 
-      super(); 
-    }
-
-    public Long doIO(Reporter reporter, 
-                       String name, 
-                       long totalSize 
-                       ) throws IOException {
-      totalSize *= MEGA;
-      
-      // create instance of local filesystem 
-      FileSystem localFS = FileSystem.getLocal(fsConfig);
-      
-      try {
-        // native runtime
-        Runtime runTime = Runtime.getRuntime();
-        
-        // copy the dso and executable from dfs
-        synchronized (this) {
-          localFS.delete(HDFS_TEST_DIR, true);
-          if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
-            throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
-          }
-        }
-        
-        synchronized (this) {
-          if (!localFS.exists(HDFS_SHLIB)) {
-            if (!FileUtil.copy(fs, HDFS_SHLIB, localFS, HDFS_SHLIB, false, fsConfig)) {
-              throw new IOException("Failed to copy " + HDFS_SHLIB + " to local filesystem");
-            }
-
-            String chmodCmd = new String(CHMOD + " a+x " + HDFS_SHLIB);
-            Process process = runTime.exec(chmodCmd);
-            int exitStatus = process.waitFor();
-            if (exitStatus != 0) {
-              throw new IOException(chmodCmd + ": Failed with exitStatus: " + exitStatus);
-            }
-          }
-        }
-        
-        synchronized (this) {
-          if (!localFS.exists(HDFS_READ)) {
-            if (!FileUtil.copy(fs, HDFS_READ, localFS, HDFS_READ, false, fsConfig)) {
-              throw new IOException("Failed to copy " + HDFS_READ + " to local filesystem");
-            }
-
-            String chmodCmd = new String(CHMOD + " a+x " + HDFS_READ); 
-            Process process = runTime.exec(chmodCmd);
-            int exitStatus = process.waitFor();
-             
-            if (exitStatus != 0) {
-              throw new IOException(chmodCmd + ": Failed with exitStatus: " + exitStatus);
-            }
-          }
-        }
-    	  	  
-        // exec the C program
-        Path inFile = new Path(DATA_DIR, name);
-        String readCmd = new String(HDFS_READ + " " + inFile + " " + totalSize + " " + 
-                                    bufferSize); 
-        Process process = runTime.exec(readCmd, null, new File(HDFS_TEST_DIR.toString()));
-        int exitStatus = process.waitFor();
-        
-        if (exitStatus != 0) {
-          throw new IOException(HDFS_READ + ": Failed with exitStatus: " + exitStatus);
-        }
-      } catch (InterruptedException interruptedException) {
-        reporter.setStatus(interruptedException.toString());
-      } finally {
-        localFS.close();
-      }
-      return new Long(totalSize);
-    }
-  }
-
-  private static void readTest(FileSystem fs) throws IOException {
-    fs.delete(READ_DIR, true);
-    runIOTest(ReadMapper.class, READ_DIR);
-  }
-
-  private static void sequentialTest(
-                                     FileSystem fs, 
-                                     int testType, 
-                                     int fileSize, 
-                                     int nrFiles
-                                     ) throws Exception {
-    IOStatMapper ioer = null;
-    if (testType == TEST_TYPE_READ)
-      ioer = new ReadMapper();
-    else if (testType == TEST_TYPE_WRITE)
-      ioer = new WriteMapper();
-    else
-      return;
-    for(int i=0; i < nrFiles; i++)
-      ioer.doIO(Reporter.NULL,
-                BASE_FILE_NAME+Integer.toString(i), 
-                MEGA*fileSize);
-  }
-
-  public static void main(String[] args) {
-    int testType = TEST_TYPE_READ;
-    int bufferSize = DEFAULT_BUFFER_SIZE;
-    int fileSize = 1;
-    int nrFiles = 1;
-    String resFileName = DEFAULT_RES_FILE_NAME;
-    boolean isSequential = false;
-
-    String version="DFSCIOTest.0.0.1";
-    String usage = "Usage: DFSCIOTest -read | -write | -clean [-nrFiles N] [-fileSize MB] [-resFile resultFileName] [-bufferSize Bytes] ";
-    
-    System.out.println(version);
-    if (args.length == 0) {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    for (int i = 0; i < args.length; i++) {       // parse command line
-      if (args[i].startsWith("-r")) {
-        testType = TEST_TYPE_READ;
-      } else if (args[i].startsWith("-w")) {
-        testType = TEST_TYPE_WRITE;
-      } else if (args[i].startsWith("-clean")) {
-        testType = TEST_TYPE_CLEANUP;
-      } else if (args[i].startsWith("-seq")) {
-        isSequential = true;
-      } else if (args[i].equals("-nrFiles")) {
-        nrFiles = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-fileSize")) {
-        fileSize = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-bufferSize")) {
-        bufferSize = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-resFile")) {
-        resFileName = args[++i];
-      }
-    }
-
-    LOG.info("nrFiles = " + nrFiles);
-    LOG.info("fileSize (MB) = " + fileSize);
-    LOG.info("bufferSize = " + bufferSize);
-  
-    try {
-      fsConfig.setInt("test.io.file.buffer.size", bufferSize);
-      FileSystem fs = FileSystem.get(fsConfig);
-      
-      if (testType != TEST_TYPE_CLEANUP) {
-        fs.delete(HDFS_TEST_DIR, true);
-        if (!fs.mkdirs(HDFS_TEST_DIR)) {
-          throw new IOException("Mkdirs failed to create " + 
-                                HDFS_TEST_DIR.toString());
-        }
-
-        //Copy the executables over to the remote filesystem
-        String hadoopHome = System.getenv("HADOOP_HOME");
-        fs.copyFromLocalFile(new Path(hadoopHome + "/libhdfs/libhdfs.so." + HDFS_LIB_VERSION),
-                             HDFS_SHLIB);
-        fs.copyFromLocalFile(new Path(hadoopHome + "/libhdfs/hdfs_read"), HDFS_READ);
-        fs.copyFromLocalFile(new Path(hadoopHome + "/libhdfs/hdfs_write"), HDFS_WRITE);
-      }
-
-      if (isSequential) {
-        long tStart = System.currentTimeMillis();
-        sequentialTest(fs, testType, fileSize, nrFiles);
-        long execTime = System.currentTimeMillis() - tStart;
-        String resultLine = "Seq Test exec time sec: " + (float)execTime / 1000;
-        LOG.info(resultLine);
-        return;
-      }
-      if (testType == TEST_TYPE_CLEANUP) {
-        cleanup(fs);
-        return;
-      }
-      createControlFile(fs, fileSize, nrFiles);
-      long tStart = System.currentTimeMillis();
-      if (testType == TEST_TYPE_WRITE)
-        writeTest(fs);
-      if (testType == TEST_TYPE_READ)
-        readTest(fs);
-      long execTime = System.currentTimeMillis() - tStart;
-    
-      analyzeResult(fs, testType, execTime, resFileName);
-    } catch(Exception e) {
-      System.err.print(e.getLocalizedMessage());
-      System.exit(-1);
-    }
-  }
-  
-  private static void analyzeResult( FileSystem fs, 
-                                     int testType,
-                                     long execTime,
-                                     String resFileName
-                                     ) throws IOException {
-    Path reduceFile;
-    if (testType == TEST_TYPE_WRITE)
-      reduceFile = new Path(WRITE_DIR, "part-00000");
-    else
-      reduceFile = new Path(READ_DIR, "part-00000");
-    DataInputStream in;
-    in = new DataInputStream(fs.open(reduceFile));
-  
-    BufferedReader lines;
-    lines = new BufferedReader(new InputStreamReader(in));
-    long tasks = 0;
-    long size = 0;
-    long time = 0;
-    float rate = 0;
-    float sqrate = 0;
-    String line;
-    while((line = lines.readLine()) != null) {
-      StringTokenizer tokens = new StringTokenizer(line, " \t\n\r\f%");
-      String attr = tokens.nextToken(); 
-      if (attr.endsWith(":tasks"))
-        tasks = Long.parseLong(tokens.nextToken());
-      else if (attr.endsWith(":size"))
-        size = Long.parseLong(tokens.	nextToken());
-      else if (attr.endsWith(":time"))
-        time = Long.parseLong(tokens.nextToken());
-      else if (attr.endsWith(":rate"))
-        rate = Float.parseFloat(tokens.nextToken());
-      else if (attr.endsWith(":sqrate"))
-        sqrate = Float.parseFloat(tokens.nextToken());
-    }
-    
-    double med = rate / 1000 / tasks;
-    double stdDev = Math.sqrt(Math.abs(sqrate / 1000 / tasks - med*med));
-    String resultLines[] = {
-      "----- DFSCIOTest ----- : " + ((testType == TEST_TYPE_WRITE) ? "write" :
-                                     (testType == TEST_TYPE_READ) ? "read" : 
-                                     "unknown"),
-      "           Date & time: " + new Date(System.currentTimeMillis()),
-      "       Number of files: " + tasks,
-      "Total MBytes processed: " + size/MEGA,
-      "     Throughput mb/sec: " + size * 1000.0 / (time * MEGA),
-      "Average IO rate mb/sec: " + med,
-      " Std IO rate deviation: " + stdDev,
-      "    Test exec time sec: " + (float)execTime / 1000,
-      "" };
-
-    PrintStream res = new PrintStream(
-                                      new FileOutputStream(
-                                                           new File(resFileName), true)); 
-    for(int i = 0; i < resultLines.length; i++) {
-      LOG.info(resultLines[i]);
-      res.println(resultLines[i]);
-    }
-  }
-
-  private static void cleanup(FileSystem fs) throws Exception {
-    LOG.info("Cleaning up test files");
-    fs.delete(new Path(TEST_ROOT_DIR), true);
-    fs.delete(HDFS_TEST_DIR, true);
-  }
-}

+ 0 - 357
src/test/hdfs-with-mr/org/apache/hadoop/fs/DistributedFSCheck.java

@@ -1,357 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
-import java.util.Date;
-import java.util.StringTokenizer;
-import java.util.TreeSet;
-import java.util.Vector;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.mapred.*;
-
-/**
- * Distributed checkup of the file system consistency.
- * <p>
- * Test file system consistency by reading each block of each file
- * of the specified file tree. 
- * Report corrupted blocks and general file statistics.
- * <p>
- * Optionally displays statistics on read performance.
- * 
- */
-public class DistributedFSCheck extends TestCase {
-  // Constants
-  private static final Log LOG = LogFactory.getLog(DistributedFSCheck.class);
-  private static final int TEST_TYPE_READ = 0;
-  private static final int TEST_TYPE_CLEANUP = 2;
-  private static final int DEFAULT_BUFFER_SIZE = 1000000;
-  private static final String DEFAULT_RES_FILE_NAME = "DistributedFSCheck_results.log";
-  private static final long MEGA = 0x100000;
-  
-  private static Configuration fsConfig = new Configuration();
-  private static Path TEST_ROOT_DIR = new Path(System.getProperty("test.build.data","/benchmarks/DistributedFSCheck"));
-  private static Path MAP_INPUT_DIR = new Path(TEST_ROOT_DIR, "map_input");
-  private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");
-
-  private FileSystem fs;
-  private long nrFiles;
-  
-  DistributedFSCheck(Configuration conf) throws Exception {
-    fsConfig = conf;
-    this.fs = FileSystem.get(conf);
-  }
-
-  /**
-   * Run distributed checkup for the entire files system.
-   * 
-   * @throws Exception
-   */
-  public void testFSBlocks() throws Exception {
-    testFSBlocks("/");
-  }
-
-  /**
-   * Run distributed checkup for the specified directory.
-   * 
-   * @param rootName root directory name
-   * @throws Exception
-   */
-  public void testFSBlocks(String rootName) throws Exception {
-    createInputFile(rootName);
-    runDistributedFSCheck();
-    cleanup();  // clean up after all to restore the system state
-  }
-
-  private void createInputFile(String rootName) throws IOException {
-    cleanup();  // clean up if previous run failed
-
-    Path inputFile = new Path(MAP_INPUT_DIR, "in_file");
-    SequenceFile.Writer writer =
-      SequenceFile.createWriter(fs, fsConfig, inputFile, 
-                                Text.class, LongWritable.class, CompressionType.NONE);
-    
-    try {
-      nrFiles = 0;
-      listSubtree(new Path(rootName), writer);
-    } finally {
-      writer.close();
-    }
-    LOG.info("Created map input files.");
-  }
-  
-  private void listSubtree(Path rootFile,
-                           SequenceFile.Writer writer
-                           ) throws IOException {
-    FileStatus rootStatus = fs.getFileStatus(rootFile);
-    listSubtree(rootStatus, writer);
-  }
-
-  private void listSubtree(FileStatus rootStatus,
-                           SequenceFile.Writer writer
-                           ) throws IOException {
-    Path rootFile = rootStatus.getPath();
-    if (!rootStatus.isDir()) {
-      nrFiles++;
-      // For a regular file generate <fName,offset> pairs
-      long blockSize = fs.getDefaultBlockSize();
-      long fileLength = rootStatus.getLen();
-      for(long offset = 0; offset < fileLength; offset += blockSize)
-        writer.append(new Text(rootFile.toString()), new LongWritable(offset));
-      return;
-    }
-    
-    FileStatus [] children = null;
-    try {
-      children = fs.listStatus(rootFile);
-    } catch (FileNotFoundException fnfe ){
-      throw new IOException("Could not get listing for " + rootFile);
-    }
-
-    for (int i = 0; i < children.length; i++)
-      listSubtree(children[i], writer);
-  }
-
-  /**
-   * DistributedFSCheck mapper class.
-   */
-  public static class DistributedFSCheckMapper extends IOMapperBase<Object> {
-
-    public DistributedFSCheckMapper() { 
-    }
-
-    public Object doIO(Reporter reporter, 
-                       String name, 
-                       long offset 
-                       ) throws IOException {
-      // open file
-      FSDataInputStream in = null;
-      try {
-        in = fs.open(new Path(name));
-      } catch(IOException e) {
-        return name + "@(missing)";
-      }
-      in.seek(offset);
-      long actualSize = 0;
-      try {
-        long blockSize = fs.getDefaultBlockSize();
-        reporter.setStatus("reading " + name + "@" + 
-                           offset + "/" + blockSize);
-        for( int curSize = bufferSize; 
-             curSize == bufferSize && actualSize < blockSize;
-             actualSize += curSize) {
-          curSize = in.read(buffer, 0, bufferSize);
-        }
-      } catch(IOException e) {
-        LOG.info("Corrupted block detected in \"" + name + "\" at " + offset);
-        return name + "@" + offset;
-      } finally {
-        in.close();
-      }
-      return new Long(actualSize);
-    }
-    
-    void collectStats(OutputCollector<Text, Text> output, 
-                      String name, 
-                      long execTime, 
-                      Object corruptedBlock) throws IOException {
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "blocks"),
-          new Text(String.valueOf(1)));
-
-      if (corruptedBlock.getClass().getName().endsWith("String")) {
-        output.collect(
-            new Text(AccumulatingReducer.VALUE_TYPE_STRING + "badBlocks"),
-            new Text((String)corruptedBlock));
-        return;
-      }
-      long totalSize = ((Long)corruptedBlock).longValue();
-      float ioRateMbSec = (float)totalSize * 1000 / (execTime * 0x100000);
-      LOG.info("Number of bytes processed = " + totalSize);
-      LOG.info("Exec time = " + execTime);
-      LOG.info("IO rate = " + ioRateMbSec);
-      
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "size"),
-          new Text(String.valueOf(totalSize)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "time"),
-          new Text(String.valueOf(execTime)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_FLOAT + "rate"),
-          new Text(String.valueOf(ioRateMbSec*1000)));
-    }
-  }
-  
-  private void runDistributedFSCheck() throws Exception {
-    JobConf job = new JobConf(fs.getConf(), DistributedFSCheck.class);
-
-    FileInputFormat.setInputPaths(job, MAP_INPUT_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(DistributedFSCheckMapper.class);
-    job.setReducerClass(AccumulatingReducer.class);
-
-    FileOutputFormat.setOutputPath(job, READ_DIR);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-  public static void main(String[] args) throws Exception {
-    int testType = TEST_TYPE_READ;
-    int bufferSize = DEFAULT_BUFFER_SIZE;
-    String resFileName = DEFAULT_RES_FILE_NAME;
-    String rootName = "/";
-    boolean viewStats = false;
-
-    String usage = "Usage: DistributedFSCheck [-root name] [-clean] [-resFile resultFileName] [-bufferSize Bytes] [-stats] ";
-    
-    if (args.length == 1 && args[0].startsWith("-h")) {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    for(int i = 0; i < args.length; i++) {       // parse command line
-      if (args[i].equals("-root")) {
-        rootName = args[++i];
-      } else if (args[i].startsWith("-clean")) {
-        testType = TEST_TYPE_CLEANUP;
-      } else if (args[i].equals("-bufferSize")) {
-        bufferSize = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-resFile")) {
-        resFileName = args[++i];
-      } else if (args[i].startsWith("-stat")) {
-        viewStats = true;
-      }
-    }
-
-    LOG.info("root = " + rootName);
-    LOG.info("bufferSize = " + bufferSize);
-  
-    Configuration conf = new Configuration();  
-    conf.setInt("test.io.file.buffer.size", bufferSize);
-    DistributedFSCheck test = new DistributedFSCheck(conf);
-
-    if (testType == TEST_TYPE_CLEANUP) {
-      test.cleanup();
-      return;
-    }
-    test.createInputFile(rootName);
-    long tStart = System.currentTimeMillis();
-    test.runDistributedFSCheck();
-    long execTime = System.currentTimeMillis() - tStart;
-    
-    test.analyzeResult(execTime, resFileName, viewStats);
-    // test.cleanup();  // clean up after all to restore the system state
-  }
-  
-  private void analyzeResult(long execTime,
-                             String resFileName,
-                             boolean viewStats
-                             ) throws IOException {
-    Path reduceFile= new Path(READ_DIR, "part-00000");
-    DataInputStream in;
-    in = new DataInputStream(fs.open(reduceFile));
-  
-    BufferedReader lines;
-    lines = new BufferedReader(new InputStreamReader(in));
-    long blocks = 0;
-    long size = 0;
-    long time = 0;
-    float rate = 0;
-    StringTokenizer  badBlocks = null;
-    long nrBadBlocks = 0;
-    String line;
-    while((line = lines.readLine()) != null) {
-      StringTokenizer tokens = new StringTokenizer(line, " \t\n\r\f%");
-      String attr = tokens.nextToken(); 
-      if (attr.endsWith("blocks"))
-        blocks = Long.parseLong(tokens.nextToken());
-      else if (attr.endsWith("size"))
-        size = Long.parseLong(tokens.nextToken());
-      else if (attr.endsWith("time"))
-        time = Long.parseLong(tokens.nextToken());
-      else if (attr.endsWith("rate"))
-        rate = Float.parseFloat(tokens.nextToken());
-      else if (attr.endsWith("badBlocks")) {
-        badBlocks = new StringTokenizer(tokens.nextToken(), ";");
-        nrBadBlocks = badBlocks.countTokens();
-      }
-    }
-    
-    Vector<String> resultLines = new Vector<String>();
-    resultLines.add( "----- DistributedFSCheck ----- : ");
-    resultLines.add( "               Date & time: " + new Date(System.currentTimeMillis()));
-    resultLines.add( "    Total number of blocks: " + blocks);
-    resultLines.add( "    Total number of  files: " + nrFiles);
-    resultLines.add( "Number of corrupted blocks: " + nrBadBlocks);
-    
-    int nrBadFilesPos = resultLines.size();
-    TreeSet<String> badFiles = new TreeSet<String>();
-    long nrBadFiles = 0;
-    if (nrBadBlocks > 0) {
-      resultLines.add("");
-      resultLines.add("----- Corrupted Blocks (file@offset) ----- : ");
-      while(badBlocks.hasMoreTokens()) {
-        String curBlock = badBlocks.nextToken();
-        resultLines.add(curBlock);
-        badFiles.add(curBlock.substring(0, curBlock.indexOf('@')));
-      }
-      nrBadFiles = badFiles.size();
-    }
-    
-    resultLines.insertElementAt(" Number of corrupted files: " + nrBadFiles, nrBadFilesPos);
-    
-    if (viewStats) {
-      resultLines.add("");
-      resultLines.add("-----   Performance  ----- : ");
-      resultLines.add("         Total MBytes read: " + size/MEGA);
-      resultLines.add("         Throughput mb/sec: " + (float)size * 1000.0 / (time * MEGA));
-      resultLines.add("    Average IO rate mb/sec: " + rate / 1000 / blocks);
-      resultLines.add("        Test exec time sec: " + (float)execTime / 1000);
-    }
-
-    PrintStream res = new PrintStream(
-                                      new FileOutputStream(
-                                                           new File(resFileName), true)); 
-    for(int i = 0; i < resultLines.size(); i++) {
-      String cur = resultLines.get(i);
-      LOG.info(cur);
-      res.println(cur);
-    }
-  }
-
-  private void cleanup() throws IOException {
-    LOG.info("Cleaning up test files");
-    fs.delete(TEST_ROOT_DIR, true);
-  }
-}

+ 0 - 125
src/test/hdfs-with-mr/org/apache/hadoop/fs/IOMapperBase.java

@@ -1,125 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
-
-/**
- * Base mapper class for IO operations.
- * <p>
- * Two abstract method {@link #doIO(Reporter, String, long)} and 
- * {@link #collectStats(OutputCollector,String,long,Object)} should be
- * overloaded in derived classes to define the IO operation and the
- * statistics data to be collected by subsequent reducers.
- * 
- */
-@SuppressWarnings("deprecation")
-public abstract class IOMapperBase<T> extends Configured
-    implements Mapper<Text, LongWritable, Text, Text> {
-  
-  protected byte[] buffer;
-  protected int bufferSize;
-  protected FileSystem fs;
-  protected String hostName;
-
-  public IOMapperBase() { 
-  }
-
-  public void configure(JobConf conf) {
-    setConf(conf);
-    try {
-      fs = FileSystem.get(conf);
-    } catch (Exception e) {
-      throw new RuntimeException("Cannot create file system.", e);
-    }
-    bufferSize = conf.getInt("test.io.file.buffer.size", 4096);
-    buffer = new byte[bufferSize];
-    try {
-      hostName = InetAddress.getLocalHost().getHostName();
-    } catch(Exception e) {
-      hostName = "localhost";
-    }
-  }
-
-  public void close() throws IOException {
-  }
-  
-  /**
-   * Perform io operation, usually read or write.
-   * 
-   * @param reporter
-   * @param name file name
-   * @param value offset within the file
-   * @return object that is passed as a parameter to 
-   *          {@link #collectStats(OutputCollector,String,long,Object)}
-   * @throws IOException
-   */
-  abstract T doIO(Reporter reporter, 
-                       String name, 
-                       long value) throws IOException;
-
-  /**
-   * Collect stat data to be combined by a subsequent reducer.
-   * 
-   * @param output
-   * @param name file name
-   * @param execTime IO execution time
-   * @param doIOReturnValue value returned by {@link #doIO(Reporter,String,long)}
-   * @throws IOException
-   */
-  abstract void collectStats(OutputCollector<Text, Text> output, 
-                             String name, 
-                             long execTime, 
-                             T doIOReturnValue) throws IOException;
-  
-  /**
-   * Map file name and offset into statistical data.
-   * <p>
-   * The map task is to get the 
-   * <tt>key</tt>, which contains the file name, and the 
-   * <tt>value</tt>, which is the offset within the file.
-   * 
-   * The parameters are passed to the abstract method 
-   * {@link #doIO(Reporter,String,long)}, which performs the io operation, 
-   * usually read or write data, and then 
-   * {@link #collectStats(OutputCollector,String,long,Object)} 
-   * is called to prepare stat data for a subsequent reducer.
-   */
-  public void map(Text key, 
-                  LongWritable value,
-                  OutputCollector<Text, Text> output, 
-                  Reporter reporter) throws IOException {
-    String name = key.toString();
-    long longValue = value.get();
-    
-    reporter.setStatus("starting " + name + " ::host = " + hostName);
-    
-    long tStart = System.currentTimeMillis();
-    T statValue = doIO(reporter, name, longValue);
-    long tEnd = System.currentTimeMillis();
-    long execTime = tEnd - tStart;
-    collectStats(output, name, execTime, statValue);
-    
-    reporter.setStatus("finished " + name + " ::host = " + hostName);
-  }
-}

+ 0 - 1129
src/test/hdfs-with-mr/org/apache/hadoop/fs/JHLogAnalyzer.java

@@ -1,1129 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.StringTokenizer;
-import java.util.HashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Job History Log Analyzer.
- * 
- * <h3>Description.</h3>
- * This a tool for parsing and analyzing history logs of map-reduce jobs.
- * History logs contain information about execution of jobs, tasks, and 
- * attempts. This tool focuses on submission, launch, start, and finish times,
- * as well as the success or failure of jobs, tasks, and attempts.
- * <p>
- * The analyzer calculates <em>per hour slot utilization</em> for the cluster 
- * as follows.
- * For each task attempt it divides the time segment from the start of the 
- * attempt t<sub>S</sub> to the finish t<sub>F</sub> into whole hours 
- * [t<sub>0</sub>, ..., t<sub>n</sub>], where t<sub>0</sub> <= t<sub>S</sub> 
- * is the maximal whole hour preceding t<sub>S</sub>, and
- * t<sub>n</sub> >= t<sub>F</sub> is the minimal whole hour after t<sub>F</sub>. 
- * Thus, [t<sub>0</sub>, ..., t<sub>n</sub>] covers the segment 
- * [t<sub>S</sub>, t<sub>F</sub>], during which the attempt was executed.
- * Each interval [t<sub>i</sub>, t<sub>i+1</sub>] fully contained in 
- * [t<sub>S</sub>, t<sub>F</sub>] corresponds to exactly one slot on
- * a map-reduce cluster (usually MAP-slot or REDUCE-slot).
- * If interval [t<sub>i</sub>, t<sub>i+1</sub>] only intersects with 
- * [t<sub>S</sub>, t<sub>F</sub>] then we say that the task 
- * attempt used just a fraction of the slot during this hour.
- * The fraction equals the size of the intersection.
- * Let slotTime(A, h) denote the number of slots calculated that way for a 
- * specific attempt A during hour h.
- * The tool then sums all slots for all attempts for every hour.
- * The result is the slot hour utilization of the cluster:
- * <tt>slotTime(h) = SUM<sub>A</sub> slotTime(A,h)</tt>.
- * <p>
- * Log analyzer calculates slot hours for <em>MAP</em> and <em>REDUCE</em> 
- * attempts separately.
- * <p>
- * Log analyzer distinguishes between <em>successful</em> and <em>failed</em>
- * attempts. Task attempt is considered successful if its own status is SUCCESS
- * and the statuses of the task and the job it is a part of are also SUCCESS.
- * Otherwise the task attempt is considered failed.
- * <p>
- * Map-reduce clusters are usually configured to have a fixed number of MAP 
- * and REDUCE slots per node. Thus the maximal possible number of slots on
- * the cluster is <tt>total_slots = total_nodes * slots_per_node</tt>.
- * Effective slot hour cannot exceed <tt>total_slots</tt> for successful
- * attempts.
- * <p>
- * <em>Pending time</em> characterizes the wait time of attempts.
- * It is calculated similarly to the slot hour except that the wait interval
- * starts when the job is submitted and ends when an attempt starts execution.
- * In addition to that pending time also includes intervals between attempts
- * of the same task if it was re-executed.
- * <p>
- * History log analyzer calculates two pending time variations. First is based
- * on job submission time as described above, second, starts the wait interval
- * when the job is launched rather than submitted.
- * 
- * <h3>Input.</h3>
- * The following input parameters can be specified in the argument string
- * to the job log analyzer:
- * <ul>
- * <li><tt>-historyDir inputDir</tt> specifies the location of the directory
- * where analyzer will be looking for job history log files.</li>
- * <li><tt>-resFile resultFile</tt> the name of the result file.</li>
- * <li><tt>-usersIncluded | -usersExcluded userList</tt> slot utilization and 
- * pending time can be calculated for all or for all but the specified users.
- * <br>
- * <tt>userList</tt> is a comma or semicolon separated list of users.</li>
- * <li><tt>-gzip</tt> is used if history log files are compressed.
- * Only {@link GzipCodec} is currently supported.</li>
- * <li><tt>-jobDelimiter pattern</tt> one can concatenate original log files into 
- * larger file(s) with the specified delimiter to recognize the end of the log
- * for one job from the next one.<br>
- * <tt>pattern</tt> is a java regular expression
- * {@link java.util.regex.Pattern}, which should match only the log delimiters.
- * <br>
- * E.g. pattern <tt>".!!FILE=.*!!"</tt> matches delimiters, which contain
- * the original history log file names in the following form:<br>
- * <tt>"$!!FILE=my.job.tracker.com_myJobId_user_wordcount.log!!"</tt></li>
- * <li><tt>-clean</tt> cleans up default directories used by the analyzer.</li>
- * <li><tt>-test</tt> test one file locally and exit;
- * does not require map-reduce.</li>
- * <li><tt>-help</tt> print usage.</li>
- * </ul>
- * 
- * <h3>Output.</h3>
- * The output file is formatted as a tab separated table consisting of four
- * columns: <tt>SERIES, PERIOD, TYPE, SLOT_HOUR</tt>.
- * <ul>
- * <li><tt>SERIES</tt> one of the four statistical series;</li>
- * <li><tt>PERIOD</tt> the start of the time interval in the following format:
- * <tt>"yyyy-mm-dd hh:mm:ss"</tt>;</li>
- * <li><tt>TYPE</tt> the slot type, e.g. MAP or REDUCE;</li>
- * <li><tt>SLOT_HOUR</tt> the value of the slot usage during this 
- * time interval.</li>
- * </ul>
- */
-@SuppressWarnings("deprecation")
-public class JHLogAnalyzer {
-  private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class);
-  // Constants
-  private static final String JHLA_ROOT_DIR = 
-                            System.getProperty("test.build.data", "stats/JHLA");
-  private static final Path INPUT_DIR = new Path(JHLA_ROOT_DIR, "jhla_input");
-  private static final String BASE_INPUT_FILE_NAME = "jhla_in_";
-  private static final Path OUTPUT_DIR = new Path(JHLA_ROOT_DIR, "jhla_output");
-  private static final Path RESULT_FILE = 
-                            new Path(JHLA_ROOT_DIR, "jhla_result.txt");
-  private static final Path DEFAULT_HISTORY_DIR = new Path("history");
-
-  private static final int DEFAULT_TIME_INTERVAL_MSEC = 1000*60*60; // 1 hour
-
-  static{
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-  }
-
-  static enum StatSeries {
-    STAT_ALL_SLOT_TIME
-          (AccumulatingReducer.VALUE_TYPE_LONG + "allSlotTime"),
-    STAT_FAILED_SLOT_TIME
-          (AccumulatingReducer.VALUE_TYPE_LONG + "failedSlotTime"),
-    STAT_SUBMIT_PENDING_SLOT_TIME
-          (AccumulatingReducer.VALUE_TYPE_LONG + "submitPendingSlotTime"),
-    STAT_LAUNCHED_PENDING_SLOT_TIME
-          (AccumulatingReducer.VALUE_TYPE_LONG + "launchedPendingSlotTime");    
-
-    private String statName = null;
-    private StatSeries(String name) {this.statName = name;}
-    public String toString() {return statName;}
-  }
-
-  private static class FileCreateDaemon extends Thread {
-    private static final int NUM_CREATE_THREADS = 10;
-    private static volatile int numFinishedThreads;
-    private static volatile int numRunningThreads;
-    private static FileStatus[] jhLogFiles;
-
-    FileSystem fs;
-    int start;
-    int end;
-
-    FileCreateDaemon(FileSystem fs, int start, int end) {
-      this.fs = fs;
-      this.start = start;
-      this.end = end;
-    }
-
-    public void run() {
-      try {
-        for(int i=start; i < end; i++) {
-          String name = getFileName(i);
-          Path controlFile = new Path(INPUT_DIR, "in_file_" + name);
-          SequenceFile.Writer writer = null;
-          try {
-            writer = SequenceFile.createWriter(fs, fs.getConf(), controlFile,
-                                               Text.class, LongWritable.class,
-                                               CompressionType.NONE);
-            String logFile = jhLogFiles[i].getPath().toString();
-            writer.append(new Text(logFile), new LongWritable(0));
-          } catch(Exception e) {
-            throw new IOException(e);
-          } finally {
-            if (writer != null)
-              writer.close();
-            writer = null;
-          }
-        }
-      } catch(IOException ex) {
-        LOG.error("FileCreateDaemon failed.", ex);
-      }
-      numFinishedThreads++;
-    }
-
-    private static void createControlFile(FileSystem fs, Path jhLogDir
-    ) throws IOException {
-      fs.delete(INPUT_DIR, true);
-      jhLogFiles = fs.listStatus(jhLogDir);
-
-      numFinishedThreads = 0;
-      try {
-        int start = 0;
-        int step = jhLogFiles.length / NUM_CREATE_THREADS
-        + ((jhLogFiles.length % NUM_CREATE_THREADS) > 0 ? 1 : 0);
-        FileCreateDaemon[] daemons = new FileCreateDaemon[NUM_CREATE_THREADS];
-        numRunningThreads = 0;
-        for(int tIdx=0; tIdx < NUM_CREATE_THREADS && start < jhLogFiles.length; tIdx++) {
-          int end = Math.min(start + step, jhLogFiles.length);
-          daemons[tIdx] = new FileCreateDaemon(fs, start, end);
-          start += step;
-          numRunningThreads++;
-        }
-        for(int tIdx=0; tIdx < numRunningThreads; tIdx++) {
-          daemons[tIdx].start();
-        }
-      } finally {
-        int prevValue = 0;
-        while(numFinishedThreads < numRunningThreads) {
-          if(prevValue < numFinishedThreads) {
-            LOG.info("Finished " + numFinishedThreads + " threads out of " + numRunningThreads);
-            prevValue = numFinishedThreads;
-          }
-          try {Thread.sleep(500);} catch (InterruptedException e) {}
-        }
-      }
-    }
-  }
-
-  private static void createControlFile(FileSystem fs, Path jhLogDir
-  ) throws IOException {
-    LOG.info("creating control file: JH log dir = " + jhLogDir);
-    FileCreateDaemon.createControlFile(fs, jhLogDir);
-    LOG.info("created control file: JH log dir = " + jhLogDir);
-  }
-
-  private static String getFileName(int fIdx) {
-    return BASE_INPUT_FILE_NAME + Integer.toString(fIdx);
-  }
-
-  /**
-   * If keyVal is of the form KEY="VALUE", then this will return [KEY, VALUE]
-   */
-  private static String [] getKeyValue(String t) throws IOException {
-    String[] keyVal = t.split("=\"*|\"");
-    return keyVal;
-  }
-
-  /**
-   * JobHistory log record.
-   */
-  private static class JobHistoryLog {
-    String JOBID;
-    String JOB_STATUS;
-    long SUBMIT_TIME;
-    long LAUNCH_TIME;
-    long FINISH_TIME;
-    long TOTAL_MAPS;
-    long TOTAL_REDUCES;
-    long FINISHED_MAPS;
-    long FINISHED_REDUCES;
-    String USER;
-    Map<String, TaskHistoryLog> tasks;
-
-    boolean isSuccessful() {
-     return (JOB_STATUS != null) && JOB_STATUS.equals("SUCCESS");
-    }
-
-    void parseLine(String line) throws IOException {
-      StringTokenizer tokens = new StringTokenizer(line);
-      if(!tokens.hasMoreTokens())
-        return;
-      String what = tokens.nextToken();
-      // Line should start with one of the following:
-      // Job, Task, MapAttempt, ReduceAttempt
-      if(what.equals("Job"))
-        updateJob(tokens);
-      else if(what.equals("Task"))
-        updateTask(tokens);
-      else if(what.indexOf("Attempt") >= 0)
-        updateTaskAttempt(tokens);
-    }
-
-    private void updateJob(StringTokenizer tokens) throws IOException {
-      while(tokens.hasMoreTokens()) {
-        String t = tokens.nextToken();
-        String[] keyVal = getKeyValue(t);
-        if(keyVal.length < 2) continue;
-
-        if(keyVal[0].equals("JOBID")) {
-          if(JOBID == null)
-            JOBID = new String(keyVal[1]);
-          else if(!JOBID.equals(keyVal[1])) {
-            LOG.error("Incorrect JOBID: "
-                + keyVal[1].substring(0, Math.min(keyVal[1].length(), 100)) 
-                + " expect " + JOBID);
-            return;
-          }
-        }
-        else if(keyVal[0].equals("JOB_STATUS"))
-          JOB_STATUS = new String(keyVal[1]);
-        else if(keyVal[0].equals("SUBMIT_TIME"))
-          SUBMIT_TIME = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("LAUNCH_TIME"))
-          LAUNCH_TIME = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("FINISH_TIME"))
-          FINISH_TIME = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("TOTAL_MAPS"))
-          TOTAL_MAPS = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("TOTAL_REDUCES"))
-          TOTAL_REDUCES = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("FINISHED_MAPS"))
-          FINISHED_MAPS = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("FINISHED_REDUCES"))
-          FINISHED_REDUCES = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("USER"))
-          USER = new String(keyVal[1]);
-      }
-    }
-
-    private void updateTask(StringTokenizer tokens) throws IOException {
-      // unpack
-      TaskHistoryLog task = new TaskHistoryLog().parse(tokens);
-      if(task.TASKID == null) {
-        LOG.error("TASKID = NULL for job " + JOBID);
-        return;
-      }
-      // update or insert
-      if(tasks == null)
-        tasks = new HashMap<String, TaskHistoryLog>((int)(TOTAL_MAPS + TOTAL_REDUCES));
-      TaskHistoryLog existing = tasks.get(task.TASKID);
-      if(existing == null)
-        tasks.put(task.TASKID, task);
-      else
-        existing.updateWith(task);
-    }
-
-    private void updateTaskAttempt(StringTokenizer tokens) throws IOException {
-      // unpack
-      TaskAttemptHistoryLog attempt = new TaskAttemptHistoryLog();
-      String taskID = attempt.parse(tokens);
-      if(taskID == null) return;
-      if(tasks == null)
-        tasks = new HashMap<String, TaskHistoryLog>((int)(TOTAL_MAPS + TOTAL_REDUCES));
-      TaskHistoryLog existing = tasks.get(taskID);
-      if(existing == null) {
-        existing = new TaskHistoryLog(taskID);
-        tasks.put(taskID, existing);
-      }
-      existing.updateWith(attempt);
-    }
-  }
-
-  /**
-   * TaskHistory log record.
-   */
-  private static class TaskHistoryLog {
-    String TASKID;
-    String TASK_TYPE;   // MAP, REDUCE, SETUP, CLEANUP
-    String TASK_STATUS;
-    long START_TIME;
-    long FINISH_TIME;
-    Map<String, TaskAttemptHistoryLog> attempts;
-
-    TaskHistoryLog() {}
-
-    TaskHistoryLog(String taskID) {
-      TASKID = taskID;
-    }
-
-    boolean isSuccessful() {
-      return (TASK_STATUS != null) && TASK_STATUS.equals("SUCCESS");
-    }
-
-    TaskHistoryLog parse(StringTokenizer tokens) throws IOException {
-      while(tokens.hasMoreTokens()) {
-        String t = tokens.nextToken();
-        String[] keyVal = getKeyValue(t);
-        if(keyVal.length < 2) continue;
-
-        if(keyVal[0].equals("TASKID")) {
-          if(TASKID == null)
-            TASKID = new String(keyVal[1]);
-          else if(!TASKID.equals(keyVal[1])) {
-            LOG.error("Incorrect TASKID: "
-                + keyVal[1].substring(0, Math.min(keyVal[1].length(), 100)) 
-                + " expect " + TASKID);
-            continue;
-          }
-        }
-        else if(keyVal[0].equals("TASK_TYPE"))
-          TASK_TYPE = new String(keyVal[1]);
-        else if(keyVal[0].equals("TASK_STATUS"))
-          TASK_STATUS = new String(keyVal[1]);
-        else if(keyVal[0].equals("START_TIME"))
-          START_TIME = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("FINISH_TIME"))
-          FINISH_TIME = Long.parseLong(keyVal[1]);
-      }
-      return this;
-    }
-
-    /**
-     * Update with non-null fields of the same task log record.
-     */
-    void updateWith(TaskHistoryLog from) throws IOException {
-      if(TASKID == null)
-        TASKID = from.TASKID;
-      else if(!TASKID.equals(from.TASKID)) {
-        throw new IOException("Incorrect TASKID: " + from.TASKID
-                            + " expect " + TASKID);
-      }
-      if(TASK_TYPE == null)
-        TASK_TYPE = from.TASK_TYPE;
-      else if(! TASK_TYPE.equals(from.TASK_TYPE)) {
-        LOG.error(
-            "Incorrect TASK_TYPE: " + from.TASK_TYPE + " expect " + TASK_TYPE
-            + " for task " + TASKID);
-        return;
-      }
-      if(from.TASK_STATUS != null)
-        TASK_STATUS = from.TASK_STATUS;
-      if(from.START_TIME > 0)
-        START_TIME = from.START_TIME;
-      if(from.FINISH_TIME > 0)
-        FINISH_TIME = from.FINISH_TIME;
-    }
-
-    /**
-     * Update with non-null fields of the task attempt log record.
-     */
-    void updateWith(TaskAttemptHistoryLog attempt) throws IOException {
-      if(attempt.TASK_ATTEMPT_ID == null) {
-        LOG.error("Unexpected TASK_ATTEMPT_ID = null for task " + TASKID);
-        return;
-      }
-      if(attempts == null)
-        attempts = new HashMap<String, TaskAttemptHistoryLog>();
-      TaskAttemptHistoryLog existing = attempts.get(attempt.TASK_ATTEMPT_ID);
-      if(existing == null)
-        attempts.put(attempt.TASK_ATTEMPT_ID, attempt);
-      else
-        existing.updateWith(attempt);
-      // update task start time
-      if(attempt.START_TIME > 0 && 
-          (this.START_TIME == 0 || this.START_TIME > attempt.START_TIME))
-        START_TIME = attempt.START_TIME;
-    }
-  }
-
-  /**
-   * TaskAttemptHistory log record.
-   */
-  private static class TaskAttemptHistoryLog {
-    String TASK_ATTEMPT_ID;
-    String TASK_STATUS; // this task attempt status
-    long START_TIME;
-    long FINISH_TIME;
-    long HDFS_BYTES_READ;
-    long HDFS_BYTES_WRITTEN;
-    long FILE_BYTES_READ;
-    long FILE_BYTES_WRITTEN;
-
-    /**
-     * Task attempt is considered successful iff all three statuses
-     * of the attempt, the task, and the job equal "SUCCESS".
-     */
-    boolean isSuccessful() {
-      return (TASK_STATUS != null) && TASK_STATUS.equals("SUCCESS");
-    }
-
-    String parse(StringTokenizer tokens) throws IOException {
-      String taskID = null;
-      while(tokens.hasMoreTokens()) {
-        String t = tokens.nextToken();
-        String[] keyVal = getKeyValue(t);
-        if(keyVal.length < 2) continue;
-
-        if(keyVal[0].equals("TASKID")) {
-          if(taskID == null)
-            taskID = new String(keyVal[1]);
-          else if(!taskID.equals(keyVal[1])) {
-            LOG.error("Incorrect TASKID: " + keyVal[1] + " expect " + taskID);
-            continue;
-          }
-        }
-        else if(keyVal[0].equals("TASK_ATTEMPT_ID")) {
-          if(TASK_ATTEMPT_ID == null)
-            TASK_ATTEMPT_ID = new String(keyVal[1]);
-          else if(!TASK_ATTEMPT_ID.equals(keyVal[1])) {
-            LOG.error("Incorrect TASKID: " + keyVal[1] + " expect " + taskID);
-            continue;
-          }
-        }
-        else if(keyVal[0].equals("TASK_STATUS"))
-          TASK_STATUS = new String(keyVal[1]);
-        else if(keyVal[0].equals("START_TIME"))
-          START_TIME = Long.parseLong(keyVal[1]);
-        else if(keyVal[0].equals("FINISH_TIME"))
-          FINISH_TIME = Long.parseLong(keyVal[1]);
-      }
-      return taskID;
-    }
-
-    /**
-     * Update with non-null fields of the same task attempt log record.
-     */
-    void updateWith(TaskAttemptHistoryLog from) throws IOException {
-      if(TASK_ATTEMPT_ID == null)
-        TASK_ATTEMPT_ID = from.TASK_ATTEMPT_ID;
-      else if(! TASK_ATTEMPT_ID.equals(from.TASK_ATTEMPT_ID)) {
-        throw new IOException(
-            "Incorrect TASK_ATTEMPT_ID: " + from.TASK_ATTEMPT_ID 
-            + " expect " + TASK_ATTEMPT_ID);
-      }
-      if(from.TASK_STATUS != null)
-        TASK_STATUS = from.TASK_STATUS;
-      if(from.START_TIME > 0)
-        START_TIME = from.START_TIME;
-      if(from.FINISH_TIME > 0)
-        FINISH_TIME = from.FINISH_TIME;
-      if(from.HDFS_BYTES_READ > 0)
-        HDFS_BYTES_READ = from.HDFS_BYTES_READ;
-      if(from.HDFS_BYTES_WRITTEN > 0)
-        HDFS_BYTES_WRITTEN = from.HDFS_BYTES_WRITTEN;
-      if(from.FILE_BYTES_READ > 0)
-        FILE_BYTES_READ = from.FILE_BYTES_READ;
-      if(from.FILE_BYTES_WRITTEN > 0)
-        FILE_BYTES_WRITTEN = from.FILE_BYTES_WRITTEN;
-    }
-  }
-
-  /**
-   * Key = statName*date-time*taskType
-   * Value = number of msec for the our
-   */
-  private static class IntervalKey {
-    static final String KEY_FIELD_DELIMITER = "*";
-    String statName;
-    String dateTime;
-    String taskType;
-
-    IntervalKey(String stat, long timeMSec, String taskType) {
-      statName = stat;
-      SimpleDateFormat dateF = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-      dateTime = dateF.format(new Date(timeMSec));
-      this.taskType = taskType;
-    }
-
-    IntervalKey(String key) {
-      StringTokenizer keyTokens = new StringTokenizer(key, KEY_FIELD_DELIMITER);
-      if(!keyTokens.hasMoreTokens()) return;
-      statName = keyTokens.nextToken();
-      if(!keyTokens.hasMoreTokens()) return;
-      dateTime = keyTokens.nextToken();
-      if(!keyTokens.hasMoreTokens()) return;
-      taskType = keyTokens.nextToken();
-    }
-
-    void setStatName(String stat) {
-      statName = stat;
-    }
-
-    String getStringKey() {
-      return statName + KEY_FIELD_DELIMITER +
-             dateTime + KEY_FIELD_DELIMITER +
-             taskType;
-    }
-
-    Text getTextKey() {
-      return new Text(getStringKey());
-    }
-
-    public String toString() {
-      return getStringKey();
-    }
-  }
-
-  /**
-   * Mapper class.
-   */
-  private static class JHLAMapper extends IOMapperBase<Object> {
-    /**
-     * A line pattern, which delimits history logs of different jobs,
-     * if multiple job logs are written in the same file.
-     * Null value means only one job log per file is expected.
-     * The pattern should be a regular expression as in
-     * {@link String#matches(String)}.
-     */
-    String jobDelimiterPattern;
-    int maxJobDelimiterLineLength;
-    /** Count only these users jobs */
-    Collection<String> usersIncluded;
-    /** Exclude jobs of the following users */
-    Collection<String> usersExcluded;
-    /** Type of compression for compressed files: gzip */
-    Class<? extends CompressionCodec> compressionClass;
-
-    JHLAMapper() throws IOException {
-    }
-
-    JHLAMapper(Configuration conf) throws IOException {
-      configure(new JobConf(conf));
-    }
-
-    public void configure(JobConf conf) {
-      super.configure(conf );
-      usersIncluded = getUserList(conf.get("jhla.users.included", null));
-      usersExcluded = getUserList(conf.get("jhla.users.excluded", null));
-      String zipClassName = conf.get("jhla.compression.class", null);
-      try {
-        compressionClass = (zipClassName == null) ? null : 
-          Class.forName(zipClassName).asSubclass(CompressionCodec.class);
-      } catch(Exception e) {
-        throw new RuntimeException("Compression codec not found: ", e);
-      }
-      jobDelimiterPattern = conf.get("jhla.job.delimiter.pattern", null);
-      maxJobDelimiterLineLength = conf.getInt("jhla.job.delimiter.length", 512);
-    }
-
-    @Override
-    public void map(Text key, 
-                    LongWritable value,
-                    OutputCollector<Text, Text> output, 
-                    Reporter reporter) throws IOException {
-      String name = key.toString();
-      long longValue = value.get();
-      
-      reporter.setStatus("starting " + name + " ::host = " + hostName);
-      
-      long tStart = System.currentTimeMillis();
-      parseLogFile(fs, new Path(name), longValue, output, reporter);
-      long tEnd = System.currentTimeMillis();
-      long execTime = tEnd - tStart;
-      
-      reporter.setStatus("finished " + name + " ::host = " + hostName +
-          " in " + execTime/1000 + " sec.");
-    }
-
-    public Object doIO(Reporter reporter, 
-                       String path, // full path of history log file 
-                       long offset  // starting offset within the file
-                       ) throws IOException {
-      return null;
-    }
-
-    void collectStats(OutputCollector<Text, Text> output, 
-        String name,
-        long execTime,
-        Object jobObjects) throws IOException {
-    }
-
-    private boolean isEndOfJobLog(String line) {
-      if(jobDelimiterPattern == null)
-        return false;
-      return line.matches(jobDelimiterPattern);
-    }
-
-    /**
-     * Collect information about one job.
-     * 
-     * @param fs - file system
-     * @param filePath - full path of a history log file
-     * @param offset - starting offset in the history log file
-     * @throws IOException
-     */
-    public void parseLogFile(FileSystem fs,
-                                    Path filePath,
-                                    long offset,
-                                    OutputCollector<Text, Text> output,
-                                    Reporter reporter
-                                  ) throws IOException {
-      InputStream in = null;
-      try {
-        // open file & seek
-        FSDataInputStream stm = fs.open(filePath);
-        stm.seek(offset);
-        in = stm;
-        LOG.info("Opened " + filePath);
-        reporter.setStatus("Opened " + filePath);
-        // get a compression filter if specified
-        if(compressionClass != null) {
-          CompressionCodec codec = (CompressionCodec)
-            ReflectionUtils.newInstance(compressionClass, new Configuration());
-          in = codec.createInputStream(stm);
-          LOG.info("Codec created " + filePath);
-          reporter.setStatus("Codec created " + filePath);
-        }
-        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-        LOG.info("Reader created " + filePath);
-        // skip to the next job log start
-        long processed = 0L;
-        if(jobDelimiterPattern != null) {
-          for(String line = reader.readLine();
-                line != null; line = reader.readLine()) {
-            if((stm.getPos() - processed) > 100000) {
-              processed = stm.getPos();
-              reporter.setStatus("Processing " + filePath + " at " + processed);
-            }
-            if(isEndOfJobLog(line))
-              break;
-          }
-        }
-        // parse lines and update job history
-        JobHistoryLog jh = new JobHistoryLog();
-        int jobLineCount = 0;
-        for(String line = readLine(reader);
-              line != null; line = readLine(reader)) {
-          jobLineCount++;
-          if((stm.getPos() - processed) > 20000) {
-            processed = stm.getPos();
-            long numTasks = (jh.tasks == null ? 0 : jh.tasks.size());
-            String txt = "Processing " + filePath + " at " + processed
-                    + " # tasks = " + numTasks;
-            reporter.setStatus(txt);
-            LOG.info(txt);
-          }
-          if(isEndOfJobLog(line)) {
-            if(jh.JOBID != null) {
-              LOG.info("Finished parsing job: " + jh.JOBID
-                     + " line count = " + jobLineCount);
-              collectJobStats(jh, output, reporter);
-              LOG.info("Collected stats for job: " + jh.JOBID);
-            }
-            jh = new JobHistoryLog();
-            jobLineCount = 0;
-          } else
-            jh.parseLine(line);
-        }
-        if(jh.JOBID == null) {
-          LOG.error("JOBID = NULL in " + filePath + " at " + processed);
-          return;
-        }
-        collectJobStats(jh, output, reporter);
-      } catch(Exception ie) {
-        // parsing errors can happen if the file has been truncated
-        LOG.error("JHLAMapper.parseLogFile", ie);
-        reporter.setStatus("JHLAMapper.parseLogFile failed "
-                          + StringUtils.stringifyException(ie));
-        throw new IOException("Job failed.", ie);
-      } finally {
-        if(in != null) in.close();
-      }
-    }
-
-    /**
-     * Read lines until one ends with a " ." or "\" "
-     */
-    private StringBuffer resBuffer = new StringBuffer();
-    private String readLine(BufferedReader reader) throws IOException {
-      resBuffer.setLength(0);
-      reader.mark(maxJobDelimiterLineLength);
-      for(String line = reader.readLine();
-                line != null; line = reader.readLine()) {
-        if(isEndOfJobLog(line)) {
-          if(resBuffer.length() == 0)
-            resBuffer.append(line);
-          else
-            reader.reset();
-          break;
-        }
-        if(resBuffer.length() == 0)
-          resBuffer.append(line);
-        else if(resBuffer.length() < 32000)
-          resBuffer.append(line);
-        if(line.endsWith(" .") || line.endsWith("\" ")) {
-          break;
-        }
-        reader.mark(maxJobDelimiterLineLength);
-      }
-      String result = resBuffer.length() == 0 ? null : resBuffer.toString();
-      resBuffer.setLength(0);
-      return result;
-    }
-
-    private void collectPerIntervalStats(OutputCollector<Text, Text> output,
-        long start, long finish, String taskType,
-        StatSeries ... stats) throws IOException {
-      long curInterval = (start / DEFAULT_TIME_INTERVAL_MSEC)
-                                * DEFAULT_TIME_INTERVAL_MSEC;
-      long curTime = start;
-      long accumTime = 0;
-      while(curTime < finish) {
-        // how much of the task time belonged to current interval
-        long nextInterval = curInterval + DEFAULT_TIME_INTERVAL_MSEC;
-        long intervalTime = ((finish < nextInterval) ? 
-            finish : nextInterval) - curTime;
-        IntervalKey key = new IntervalKey("", curInterval, taskType);
-        Text val = new Text(String.valueOf(intervalTime));
-        for(StatSeries statName : stats) {
-          key.setStatName(statName.toString());
-          output.collect(key.getTextKey(), val);
-        }
-
-        curTime = curInterval = nextInterval;
-        accumTime += intervalTime;
-      }
-      // For the pending stat speculative attempts may intersect.
-      // Only one of them is considered pending.
-      assert accumTime == finish - start || finish < start;
-    }
-
-    private void collectJobStats(JobHistoryLog jh,
-                                        OutputCollector<Text, Text> output,
-                                        Reporter reporter
-                                        ) throws IOException {
-      if(jh == null)
-        return;
-      if(jh.tasks == null)
-        return;
-      if(jh.SUBMIT_TIME <= 0)
-        throw new IOException("Job " + jh.JOBID 
-                            + " SUBMIT_TIME = " + jh.SUBMIT_TIME);
-      if(usersIncluded != null && !usersIncluded.contains(jh.USER))
-          return;
-      if(usersExcluded != null && usersExcluded.contains(jh.USER))
-          return;
-
-      int numAttempts = 0;
-      long totalTime = 0;
-      boolean jobSuccess = jh.isSuccessful();
-      long jobWaitTime = jh.LAUNCH_TIME - jh.SUBMIT_TIME;
-      // attemptSubmitTime is the job's SUBMIT_TIME,
-      // or the previous attempt FINISH_TIME for all subsequent attempts
-      for(TaskHistoryLog th : jh.tasks.values()) {
-        if(th.attempts == null)
-          continue;
-        // Task is successful iff both the task and the job are a "SUCCESS"
-        long attemptSubmitTime = jh.LAUNCH_TIME;
-        boolean taskSuccess = jobSuccess && th.isSuccessful();
-        for(TaskAttemptHistoryLog tah : th.attempts.values()) {
-          // Task attempt is considered successful iff all three statuses
-          // of the attempt, the task, and the job equal "SUCCESS"
-          boolean success = taskSuccess && tah.isSuccessful();
-          if(tah.START_TIME == 0) {
-            LOG.error("Start time 0 for task attempt " + tah.TASK_ATTEMPT_ID);
-            continue;
-          }
-          if(tah.FINISH_TIME < tah.START_TIME) {
-            LOG.error("Finish time " + tah.FINISH_TIME + " is less than " +
-            		"Start time " + tah.START_TIME + " for task attempt " +
-            		tah.TASK_ATTEMPT_ID);
-            tah.FINISH_TIME = tah.START_TIME;
-          }
-
-          if(!"MAP".equals(th.TASK_TYPE) && !"REDUCE".equals(th.TASK_TYPE) &&
-             !"CLEANUP".equals(th.TASK_TYPE) && !"SETUP".equals(th.TASK_TYPE)) {
-            LOG.error("Unexpected TASK_TYPE = " + th.TASK_TYPE
-            + " for attempt " + tah.TASK_ATTEMPT_ID);
-          }
-
-          collectPerIntervalStats(output,
-                  attemptSubmitTime, tah.START_TIME, th.TASK_TYPE,
-                  StatSeries.STAT_LAUNCHED_PENDING_SLOT_TIME);
-          collectPerIntervalStats(output,
-                  attemptSubmitTime - jobWaitTime, tah.START_TIME, th.TASK_TYPE,
-                  StatSeries.STAT_SUBMIT_PENDING_SLOT_TIME);
-          if(success)
-            collectPerIntervalStats(output,
-                  tah.START_TIME, tah.FINISH_TIME, th.TASK_TYPE,
-                  StatSeries.STAT_ALL_SLOT_TIME);
-          else
-            collectPerIntervalStats(output,
-                  tah.START_TIME, tah.FINISH_TIME, th.TASK_TYPE,
-                  StatSeries.STAT_ALL_SLOT_TIME,
-                  StatSeries.STAT_FAILED_SLOT_TIME);
-          totalTime += (tah.FINISH_TIME - tah.START_TIME);
-          numAttempts++;
-          if(numAttempts % 500 == 0) {
-            reporter.setStatus("Processing " + jh.JOBID + " at " + numAttempts);
-          }
-          attemptSubmitTime = tah.FINISH_TIME;
-        }
-      }
-      LOG.info("Total    Maps = " + jh.TOTAL_MAPS
-          + "  Reduces = " + jh.TOTAL_REDUCES);
-      LOG.info("Finished Maps = " + jh.FINISHED_MAPS
-          + "  Reduces = " + jh.FINISHED_REDUCES);
-      LOG.info("numAttempts = " + numAttempts);
-      LOG.info("totalTime   = " + totalTime);
-      LOG.info("averageAttemptTime = " 
-          + (numAttempts==0 ? 0 : totalTime/numAttempts));
-      LOG.info("jobTotalTime = " + (jh.FINISH_TIME <= jh.SUBMIT_TIME? 0 :
-                                    jh.FINISH_TIME - jh.SUBMIT_TIME));
-    }
-  }
-
-  public static class JHLAPartitioner implements Partitioner<Text, Text> {
-    static final int NUM_REDUCERS = 9;
-
-    public void configure(JobConf conf) {}
-
-    public int getPartition(Text key, Text value, int numPartitions) {
-      IntervalKey intKey = new IntervalKey(key.toString());
-      if(intKey.statName.equals(StatSeries.STAT_ALL_SLOT_TIME.toString())) {
-        if(intKey.taskType.equals("MAP"))
-          return 0;
-        else if(intKey.taskType.equals("REDUCE"))
-          return 1;
-      } else if(intKey.statName.equals(
-          StatSeries.STAT_SUBMIT_PENDING_SLOT_TIME.toString())) {
-        if(intKey.taskType.equals("MAP"))
-          return 2;
-        else if(intKey.taskType.equals("REDUCE"))
-          return 3;
-      } else if(intKey.statName.equals(
-          StatSeries.STAT_LAUNCHED_PENDING_SLOT_TIME.toString())) {
-        if(intKey.taskType.equals("MAP"))
-          return 4;
-        else if(intKey.taskType.equals("REDUCE"))
-          return 5;
-      } else if(intKey.statName.equals(
-          StatSeries.STAT_FAILED_SLOT_TIME.toString())) {
-        if(intKey.taskType.equals("MAP"))
-          return 6;
-        else if(intKey.taskType.equals("REDUCE"))
-          return 7;
-      }
-      return 8;
-    }
-  }
-
-  private static void runJHLA(
-          Class<? extends Mapper<Text, LongWritable, Text, Text>> mapperClass, 
-          Path outputDir,
-          Configuration fsConfig) throws IOException {
-    JobConf job = new JobConf(fsConfig, JHLogAnalyzer.class);
-
-    job.setPartitionerClass(JHLAPartitioner.class);
-
-    FileInputFormat.setInputPaths(job, INPUT_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(mapperClass);
-    job.setReducerClass(AccumulatingReducer.class);
-
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setNumReduceTasks(JHLAPartitioner.NUM_REDUCERS);
-    JobClient.runJob(job);
-  }
-
-  private static class LoggingCollector implements OutputCollector<Text, Text> {
-    public void collect(Text key, Text value) throws IOException {
-      LOG.info(key + " == " + value);
-    }
-  }
-
-  /**
-   * Run job history log analyser.
-   */
-  public static void main(String[] args) {
-    Path resFileName = RESULT_FILE;
-    Configuration conf = new Configuration();
-
-    try {
-      conf.setInt("test.io.file.buffer.size", 0);
-      Path historyDir = DEFAULT_HISTORY_DIR;
-      String testFile = null;
-      boolean cleanup = false;
-
-      boolean initControlFiles = true;
-      for (int i = 0; i < args.length; i++) {       // parse command line
-        if (args[i].equalsIgnoreCase("-historyDir")) {
-          historyDir = new Path(args[++i]);
-        } else if (args[i].equalsIgnoreCase("-resFile")) {
-          resFileName = new Path(args[++i]);
-        } else if (args[i].equalsIgnoreCase("-usersIncluded")) {
-          conf.set("jhla.users.included", args[++i]);
-        } else if (args[i].equalsIgnoreCase("-usersExcluded")) {
-          conf.set("jhla.users.excluded", args[++i]);
-        } else if (args[i].equalsIgnoreCase("-gzip")) {
-          conf.set("jhla.compression.class", GzipCodec.class.getCanonicalName());
-        } else if (args[i].equalsIgnoreCase("-jobDelimiter")) {
-          conf.set("jhla.job.delimiter.pattern", args[++i]);
-        } else if (args[i].equalsIgnoreCase("-jobDelimiterLength")) {
-          conf.setInt("jhla.job.delimiter.length", Integer.parseInt(args[++i]));
-        } else if(args[i].equalsIgnoreCase("-noInit")) {
-          initControlFiles = false;
-        } else if(args[i].equalsIgnoreCase("-test")) {
-          testFile = args[++i];
-        } else if(args[i].equalsIgnoreCase("-clean")) {
-          cleanup = true;
-        } else if(args[i].equalsIgnoreCase("-jobQueue")) {
-          conf.set("mapred.job.queue.name", args[++i]);
-        } else if(args[i].startsWith("-Xmx")) {
-          conf.set("mapred.child.java.opts", args[i]);
-        } else {
-          printUsage();
-        }
-      }
-
-      if(cleanup) {
-        cleanup(conf);
-        return;
-      }
-      if(testFile != null) {
-        LOG.info("Start JHLA test ============ ");
-        LocalFileSystem lfs = FileSystem.getLocal(conf);
-        conf.set("fs.default.name", "file:///");
-        JHLAMapper map = new JHLAMapper(conf);
-        map.parseLogFile(lfs, new Path(testFile), 0L,
-                         new LoggingCollector(), Reporter.NULL);
-        return;
-      }
-
-      FileSystem fs = FileSystem.get(conf);
-      if(initControlFiles)
-        createControlFile(fs, historyDir);
-      long tStart = System.currentTimeMillis();
-      runJHLA(JHLAMapper.class, OUTPUT_DIR, conf);
-      long execTime = System.currentTimeMillis() - tStart;
-
-      analyzeResult(fs, 0, execTime, resFileName);
-    } catch(IOException e) {
-      System.err.print(StringUtils.stringifyException(e));
-      System.exit(-1);
-    }
-  }
-
-
-  private static void printUsage() {
-    String className = JHLogAnalyzer.class.getSimpleName();
-    System.err.println("Usage: " + className
-      + "\n\t[-historyDir inputDir] | [-resFile resultFile] |"
-      + "\n\t[-usersIncluded | -usersExcluded userList] |"
-      + "\n\t[-gzip] | [-jobDelimiter pattern] |"
-      + "\n\t[-help | -clean | -test testFile]");
-    System.exit(-1);
-  }
-
-  private static Collection<String> getUserList(String users) {
-    if(users == null)
-      return null;
-    StringTokenizer tokens = new StringTokenizer(users, ",;");
-    Collection<String> userList = new ArrayList<String>(tokens.countTokens());
-    while(tokens.hasMoreTokens())
-      userList.add(tokens.nextToken());
-    return userList;
-  }
-
-  /**
-   * Result is combined from all reduce output files and is written to
-   * RESULT_FILE in the format
-   * column 1: 
-   */
-  private static void analyzeResult( FileSystem fs, 
-                                     int testType,
-                                     long execTime,
-                                     Path resFileName
-                                     ) throws IOException {
-    LOG.info("Analizing results ...");
-    DataOutputStream out = null;
-    BufferedWriter writer = null;
-    try {
-      out = new DataOutputStream(fs.create(resFileName));
-      writer = new BufferedWriter(new OutputStreamWriter(out));
-      writer.write("SERIES\tPERIOD\tTYPE\tSLOT_HOUR\n");
-      FileStatus[] reduceFiles = fs.listStatus(OUTPUT_DIR);
-      assert reduceFiles.length == JHLAPartitioner.NUM_REDUCERS;
-      for(int i = 0; i < JHLAPartitioner.NUM_REDUCERS; i++) {
-        DataInputStream in = null;
-        BufferedReader lines = null;
-        try {
-          in = fs.open(reduceFiles[i].getPath());
-          lines = new BufferedReader(new InputStreamReader(in));
-    
-          String line;
-          while((line = lines.readLine()) != null) {
-            StringTokenizer tokens = new StringTokenizer(line, "\t*");
-            String attr = tokens.nextToken();
-            String dateTime = tokens.nextToken();
-            String taskType = tokens.nextToken();
-            double val = Long.parseLong(tokens.nextToken()) /
-                                    (double)DEFAULT_TIME_INTERVAL_MSEC;
-            writer.write(attr.substring(2));  // skip the stat type "l:"
-            writer.write("\t");
-            writer.write(dateTime);
-            writer.write("\t");
-            writer.write(taskType);
-            writer.write("\t");
-            writer.write(String.valueOf((float)val));
-            writer.newLine();
-          }
-        } finally {
-          if(lines != null) lines.close();
-          if(in != null) in.close();
-        }
-      }
-    } finally {
-      if(writer != null) writer.close();
-      if(out != null) out.close();
-    }
-    LOG.info("Analizing results ... done.");
-  }
-
-  private static void cleanup(Configuration conf) throws IOException {
-    LOG.info("Cleaning up test files");
-    FileSystem fs = FileSystem.get(conf);
-    fs.delete(new Path(JHLA_ROOT_DIR), true);
-  }
-}

+ 0 - 456
src/test/hdfs-with-mr/org/apache/hadoop/fs/TestDFSIO.java

@@ -1,456 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.util.Date;
-import java.util.StringTokenizer;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Distributed i/o benchmark.
- * <p>
- * This test writes into or reads from a specified number of files.
- * File size is specified as a parameter to the test. 
- * Each file is accessed in a separate map task.
- * <p>
- * The reducer collects the following statistics:
- * <ul>
- * <li>number of tasks completed</li>
- * <li>number of bytes written/read</li>
- * <li>execution time</li>
- * <li>io rate</li>
- * <li>io rate squared</li>
- * </ul>
- *    
- * Finally, the following information is appended to a local file
- * <ul>
- * <li>read or write test</li>
- * <li>date and time the test finished</li>   
- * <li>number of files</li>
- * <li>total number of bytes processed</li>
- * <li>throughput in mb/sec (total number of bytes / sum of processing times)</li>
- * <li>average i/o rate in mb/sec per file</li>
- * <li>standard deviation of i/o rate </li>
- * </ul>
- */
-public class TestDFSIO extends TestCase {
-  // Constants
-  private static final Log LOG = LogFactory.getLog(TestDFSIO.class);
-  private static final int TEST_TYPE_READ = 0;
-  private static final int TEST_TYPE_WRITE = 1;
-  private static final int TEST_TYPE_CLEANUP = 2;
-  private static final int DEFAULT_BUFFER_SIZE = 1000000;
-  private static final String BASE_FILE_NAME = "test_io_";
-  private static final String DEFAULT_RES_FILE_NAME = "TestDFSIO_results.log";
-  
-  private static final long MEGA = 0x100000;
-  private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/TestDFSIO");
-  private static Path CONTROL_DIR = new Path(TEST_ROOT_DIR, "io_control");
-  private static Path WRITE_DIR = new Path(TEST_ROOT_DIR, "io_write");
-  private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");
-  private static Path DATA_DIR = new Path(TEST_ROOT_DIR, "io_data");
-
-  static{
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-  }
-
-  /**
-   * Run the test with default parameters.
-   * 
-   * @throws Exception
-   */
-  public void testIOs() throws Exception {
-    testIOs(10, 10, new Configuration());
-  }
-
-  /**
-   * Run the test with the specified parameters.
-   * 
-   * @param fileSize file size
-   * @param nrFiles number of files
-   * @throws IOException
-   */
-  public static void testIOs(int fileSize, int nrFiles, Configuration fsConfig)
-    throws IOException {
-
-    FileSystem fs = FileSystem.get(fsConfig);
-
-    createControlFile(fs, fileSize, nrFiles, fsConfig);
-    writeTest(fs, fsConfig);
-    readTest(fs, fsConfig);
-    cleanup(fs);
-  }
-
-  private static void createControlFile(FileSystem fs,
-                                        int fileSize, // in MB 
-                                        int nrFiles,
-                                        Configuration fsConfig
-                                        ) throws IOException {
-    LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
-
-    fs.delete(CONTROL_DIR, true);
-
-    for(int i=0; i < nrFiles; i++) {
-      String name = getFileName(i);
-      Path controlFile = new Path(CONTROL_DIR, "in_file_" + name);
-      SequenceFile.Writer writer = null;
-      try {
-        writer = SequenceFile.createWriter(fs, fsConfig, controlFile,
-                                           Text.class, LongWritable.class,
-                                           CompressionType.NONE);
-        writer.append(new Text(name), new LongWritable(fileSize));
-      } catch(Exception e) {
-        throw new IOException(e.getLocalizedMessage());
-      } finally {
-    	if (writer != null)
-          writer.close();
-    	writer = null;
-      }
-    }
-    LOG.info("created control files for: "+nrFiles+" files");
-  }
-
-  private static String getFileName(int fIdx) {
-    return BASE_FILE_NAME + Integer.toString(fIdx);
-  }
-  
-  /**
-   * Write/Read mapper base class.
-   * <p>
-   * Collects the following statistics per task:
-   * <ul>
-   * <li>number of tasks completed</li>
-   * <li>number of bytes written/read</li>
-   * <li>execution time</li>
-   * <li>i/o rate</li>
-   * <li>i/o rate squared</li>
-   * </ul>
-   */
-  private abstract static class IOStatMapper<T> extends IOMapperBase<T> {
-    IOStatMapper() { 
-    }
-    
-    void collectStats(OutputCollector<Text, Text> output, 
-                      String name,
-                      long execTime, 
-                      Long objSize) throws IOException {
-      long totalSize = objSize.longValue();
-      float ioRateMbSec = (float)totalSize * 1000 / (execTime * MEGA);
-      LOG.info("Number of bytes processed = " + totalSize);
-      LOG.info("Exec time = " + execTime);
-      LOG.info("IO rate = " + ioRateMbSec);
-      
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "tasks"),
-          new Text(String.valueOf(1)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "size"),
-          new Text(String.valueOf(totalSize)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_LONG + "time"),
-          new Text(String.valueOf(execTime)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_FLOAT + "rate"),
-          new Text(String.valueOf(ioRateMbSec*1000)));
-      output.collect(new Text(AccumulatingReducer.VALUE_TYPE_FLOAT + "sqrate"),
-          new Text(String.valueOf(ioRateMbSec*ioRateMbSec*1000)));
-    }
-  }
-
-  /**
-   * Write mapper class.
-   */
-  public static class WriteMapper extends IOStatMapper<Long> {
-
-    public WriteMapper() { 
-      for(int i=0; i < bufferSize; i++)
-        buffer[i] = (byte)('0' + i % 50);
-    }
-
-    public Long doIO(Reporter reporter, 
-                       String name, 
-                       long totalSize 
-                       ) throws IOException {
-      // create file
-      totalSize *= MEGA;
-      OutputStream out;
-      out = fs.create(new Path(DATA_DIR, name), true, bufferSize);
-      
-      try {
-        // write to the file
-        long nrRemaining;
-        for (nrRemaining = totalSize; nrRemaining > 0; nrRemaining -= bufferSize) {
-          int curSize = (bufferSize < nrRemaining) ? bufferSize : (int)nrRemaining; 
-          out.write(buffer, 0, curSize);
-          reporter.setStatus("writing " + name + "@" + 
-                             (totalSize - nrRemaining) + "/" + totalSize 
-                             + " ::host = " + hostName);
-        }
-      } finally {
-        out.close();
-      }
-      return Long.valueOf(totalSize);
-    }
-  }
-
-  private static void writeTest(FileSystem fs, Configuration fsConfig)
-  throws IOException {
-
-    fs.delete(DATA_DIR, true);
-    fs.delete(WRITE_DIR, true);
-    
-    runIOTest(WriteMapper.class, WRITE_DIR, fsConfig);
-  }
-  
-  @SuppressWarnings("deprecation")
-  private static void runIOTest(
-          Class<? extends Mapper<Text, LongWritable, Text, Text>> mapperClass, 
-          Path outputDir,
-          Configuration fsConfig) throws IOException {
-    JobConf job = new JobConf(fsConfig, TestDFSIO.class);
-
-    FileInputFormat.setInputPaths(job, CONTROL_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(mapperClass);
-    job.setReducerClass(AccumulatingReducer.class);
-
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-  /**
-   * Read mapper class.
-   */
-  public static class ReadMapper extends IOStatMapper<Long> {
-
-    public ReadMapper() { 
-    }
-
-    public Long doIO(Reporter reporter, 
-                       String name, 
-                       long totalSize 
-                       ) throws IOException {
-      totalSize *= MEGA;
-      // open file
-      DataInputStream in = fs.open(new Path(DATA_DIR, name));
-      try {
-        long actualSize = 0;
-        for(int curSize = bufferSize; curSize == bufferSize;) {
-          curSize = in.read(buffer, 0, bufferSize);
-          actualSize += curSize;
-          reporter.setStatus("reading " + name + "@" + 
-                             actualSize + "/" + totalSize 
-                             + " ::host = " + hostName);
-        }
-      } finally {
-        in.close();
-      }
-      return Long.valueOf(totalSize);
-    }
-  }
-
-  private static void readTest(FileSystem fs, Configuration fsConfig)
-  throws IOException {
-    fs.delete(READ_DIR, true);
-    runIOTest(ReadMapper.class, READ_DIR, fsConfig);
-  }
-
-  private static void sequentialTest(FileSystem fs, 
-                                     int testType, 
-                                     int fileSize, 
-                                     int nrFiles
-                                     ) throws Exception {
-    IOStatMapper<Long> ioer = null;
-    if (testType == TEST_TYPE_READ)
-      ioer = new ReadMapper();
-    else if (testType == TEST_TYPE_WRITE)
-      ioer = new WriteMapper();
-    else
-      return;
-    for(int i=0; i < nrFiles; i++)
-      ioer.doIO(Reporter.NULL,
-                BASE_FILE_NAME+Integer.toString(i), 
-                MEGA*fileSize);
-  }
-
-  public static void main(String[] args) {
-    int testType = TEST_TYPE_READ;
-    int bufferSize = DEFAULT_BUFFER_SIZE;
-    int fileSize = 1;
-    int nrFiles = 1;
-    String resFileName = DEFAULT_RES_FILE_NAME;
-    boolean isSequential = false;
-    
-    String className = TestDFSIO.class.getSimpleName();
-    String version = className + ".0.0.4";
-    String usage = "Usage: " + className + " -read | -write | -clean [-nrFiles N] [-fileSize MB] [-resFile resultFileName] [-bufferSize Bytes] ";
-    
-    System.out.println(version);
-    if (args.length == 0) {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    for (int i = 0; i < args.length; i++) {       // parse command line
-      if (args[i].startsWith("-read")) {
-        testType = TEST_TYPE_READ;
-      } else if (args[i].equals("-write")) {
-        testType = TEST_TYPE_WRITE;
-      } else if (args[i].equals("-clean")) {
-        testType = TEST_TYPE_CLEANUP;
-      } else if (args[i].startsWith("-seq")) {
-        isSequential = true;
-      } else if (args[i].equals("-nrFiles")) {
-        nrFiles = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-fileSize")) {
-        fileSize = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-bufferSize")) {
-        bufferSize = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-resFile")) {
-        resFileName = args[++i];
-      }
-    }
-
-    LOG.info("nrFiles = " + nrFiles);
-    LOG.info("fileSize (MB) = " + fileSize);
-    LOG.info("bufferSize = " + bufferSize);
-  
-    try {
-      Configuration fsConfig = new Configuration();
-      fsConfig.setInt("test.io.file.buffer.size", bufferSize);
-      FileSystem fs = FileSystem.get(fsConfig);
-
-      if (isSequential) {
-        long tStart = System.currentTimeMillis();
-        sequentialTest(fs, testType, fileSize, nrFiles);
-        long execTime = System.currentTimeMillis() - tStart;
-        String resultLine = "Seq Test exec time sec: " + (float)execTime / 1000;
-        LOG.info(resultLine);
-        return;
-      }
-      if (testType == TEST_TYPE_CLEANUP) {
-        cleanup(fs);
-        return;
-      }
-      createControlFile(fs, fileSize, nrFiles, fsConfig);
-      long tStart = System.currentTimeMillis();
-      if (testType == TEST_TYPE_WRITE)
-        writeTest(fs, fsConfig);
-      if (testType == TEST_TYPE_READ)
-        readTest(fs, fsConfig);
-      long execTime = System.currentTimeMillis() - tStart;
-    
-      analyzeResult(fs, testType, execTime, resFileName);
-    } catch(Exception e) {
-      System.err.print(StringUtils.stringifyException(e));
-      System.exit(-1);
-    }
-  }
-  
-  private static void analyzeResult( FileSystem fs, 
-                                     int testType,
-                                     long execTime,
-                                     String resFileName
-                                     ) throws IOException {
-    Path reduceFile;
-    if (testType == TEST_TYPE_WRITE)
-      reduceFile = new Path(WRITE_DIR, "part-00000");
-    else
-      reduceFile = new Path(READ_DIR, "part-00000");
-    long tasks = 0;
-    long size = 0;
-    long time = 0;
-    float rate = 0;
-    float sqrate = 0;
-    DataInputStream in = null;
-    BufferedReader lines = null;
-    try {
-      in = new DataInputStream(fs.open(reduceFile));
-      lines = new BufferedReader(new InputStreamReader(in));
-      String line;
-      while((line = lines.readLine()) != null) {
-        StringTokenizer tokens = new StringTokenizer(line, " \t\n\r\f%");
-        String attr = tokens.nextToken(); 
-        if (attr.endsWith(":tasks"))
-          tasks = Long.parseLong(tokens.nextToken());
-        else if (attr.endsWith(":size"))
-          size = Long.parseLong(tokens.nextToken());
-        else if (attr.endsWith(":time"))
-          time = Long.parseLong(tokens.nextToken());
-        else if (attr.endsWith(":rate"))
-          rate = Float.parseFloat(tokens.nextToken());
-        else if (attr.endsWith(":sqrate"))
-          sqrate = Float.parseFloat(tokens.nextToken());
-      }
-    } finally {
-      if(in != null) in.close();
-      if(lines != null) lines.close();
-    }
-    
-    double med = rate / 1000 / tasks;
-    double stdDev = Math.sqrt(Math.abs(sqrate / 1000 / tasks - med*med));
-    String resultLines[] = {
-      "----- TestDFSIO ----- : " + ((testType == TEST_TYPE_WRITE) ? "write" :
-                                    (testType == TEST_TYPE_READ) ? "read" : 
-                                    "unknown"),
-      "           Date & time: " + new Date(System.currentTimeMillis()),
-      "       Number of files: " + tasks,
-      "Total MBytes processed: " + size/MEGA,
-      "     Throughput mb/sec: " + size * 1000.0 / (time * MEGA),
-      "Average IO rate mb/sec: " + med,
-      " IO rate std deviation: " + stdDev,
-      "    Test exec time sec: " + (float)execTime / 1000,
-      "" };
-
-    PrintStream res = null;
-    try {
-      res = new PrintStream(new FileOutputStream(new File(resFileName), true)); 
-      for(int i = 0; i < resultLines.length; i++) {
-        LOG.info(resultLines[i]);
-        res.println(resultLines[i]);
-      }
-    } finally {
-      if(res != null) res.close();
-    }
-  }
-
-  private static void cleanup(FileSystem fs) throws IOException {
-    LOG.info("Cleaning up test files");
-    fs.delete(new Path(TEST_ROOT_DIR), true);
-  }
-}

+ 0 - 675
src/test/hdfs-with-mr/org/apache/hadoop/fs/TestFileSystem.java

@@ -1,675 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Random;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.HashMap;
-import java.net.InetSocketAddress;
-import java.net.URI;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.fs.shell.CommandFormat;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapred.lib.LongSumReducer;
-import org.apache.hadoop.security.UnixUserGroupInformation;
-
-public class TestFileSystem extends TestCase {
-  private static final Log LOG = FileSystem.LOG;
-
-  private static Configuration conf = new Configuration();
-  private static int BUFFER_SIZE = conf.getInt("io.file.buffer.size", 4096);
-
-  private static final long MEGA = 1024 * 1024;
-  private static final int SEEKS_PER_FILE = 4;
-
-  private static String ROOT = System.getProperty("test.build.data","fs_test");
-  private static Path CONTROL_DIR = new Path(ROOT, "fs_control");
-  private static Path WRITE_DIR = new Path(ROOT, "fs_write");
-  private static Path READ_DIR = new Path(ROOT, "fs_read");
-  private static Path DATA_DIR = new Path(ROOT, "fs_data");
-
-  public void testFs() throws Exception {
-    testFs(10 * MEGA, 100, 0);
-  }
-
-  public static void testFs(long megaBytes, int numFiles, long seed)
-    throws Exception {
-
-    FileSystem fs = FileSystem.get(conf);
-
-    if (seed == 0)
-      seed = new Random().nextLong();
-
-    LOG.info("seed = "+seed);
-
-    createControlFile(fs, megaBytes, numFiles, seed);
-    writeTest(fs, false);
-    readTest(fs, false);
-    seekTest(fs, false);
-    fs.delete(CONTROL_DIR, true);
-    fs.delete(DATA_DIR, true);
-    fs.delete(WRITE_DIR, true);
-    fs.delete(READ_DIR, true);
-  }
-
-  public static void testCommandFormat() throws Exception {
-    // This should go to TestFsShell.java when it is added.
-    CommandFormat cf;
-    cf= new CommandFormat("copyToLocal", 2,2,"crc","ignoreCrc");
-    assertEquals(cf.parse(new String[] {"-get","file", "-"}, 1).get(1), "-");
-    assertEquals(cf.parse(new String[] {"-get","file","-ignoreCrc","/foo"}, 1).get(1),"/foo");
-    cf = new CommandFormat("tail", 1, 1, "f");
-    assertEquals(cf.parse(new String[] {"-tail","fileName"}, 1).get(0),"fileName");
-    assertEquals(cf.parse(new String[] {"-tail","-f","fileName"}, 1).get(0),"fileName");
-    cf = new CommandFormat("setrep", 2, 2, "R", "w");
-    assertEquals(cf.parse(new String[] {"-setrep","-R","2","/foo/bar"}, 1).get(1), "/foo/bar");
-    cf = new CommandFormat("put", 2, 10000);
-    assertEquals(cf.parse(new String[] {"-put", "-", "dest"}, 1).get(1), "dest"); 
-  }
-
-  public static void createControlFile(FileSystem fs,
-                                       long megaBytes, int numFiles,
-                                       long seed) throws Exception {
-
-    LOG.info("creating control file: "+megaBytes+" bytes, "+numFiles+" files");
-
-    Path controlFile = new Path(CONTROL_DIR, "files");
-    fs.delete(controlFile, true);
-    Random random = new Random(seed);
-
-    SequenceFile.Writer writer =
-      SequenceFile.createWriter(fs, conf, controlFile, 
-                                Text.class, LongWritable.class, CompressionType.NONE);
-
-    long totalSize = 0;
-    long maxSize = ((megaBytes / numFiles) * 2) + 1;
-    try {
-      while (totalSize < megaBytes) {
-        Text name = new Text(Long.toString(random.nextLong()));
-
-        long size = random.nextLong();
-        if (size < 0)
-          size = -size;
-        size = size % maxSize;
-
-        //LOG.info(" adding: name="+name+" size="+size);
-
-        writer.append(name, new LongWritable(size));
-
-        totalSize += size;
-      }
-    } finally {
-      writer.close();
-    }
-    LOG.info("created control file for: "+totalSize+" bytes");
-  }
-
-  public static class WriteMapper extends Configured
-      implements Mapper<Text, LongWritable, Text, LongWritable> {
-    
-    private Random random = new Random();
-    private byte[] buffer = new byte[BUFFER_SIZE];
-    private FileSystem fs;
-    private boolean fastCheck;
-
-    // a random suffix per task
-    private String suffix = "-"+random.nextLong();
-    
-    {
-      try {
-        fs = FileSystem.get(conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    public WriteMapper() { super(null); }
-    
-    public WriteMapper(Configuration conf) { super(conf); }
-
-    public void configure(JobConf job) {
-      setConf(job);
-      fastCheck = job.getBoolean("fs.test.fastCheck", false);
-    }
-
-    public void map(Text key, LongWritable value,
-                    OutputCollector<Text, LongWritable> collector,
-                    Reporter reporter)
-      throws IOException {
-      
-      String name = key.toString();
-      long size = value.get();
-      long seed = Long.parseLong(name);
-
-      random.setSeed(seed);
-      reporter.setStatus("creating " + name);
-
-      // write to temp file initially to permit parallel execution
-      Path tempFile = new Path(DATA_DIR, name+suffix);
-      OutputStream out = fs.create(tempFile);
-
-      long written = 0;
-      try {
-        while (written < size) {
-          if (fastCheck) {
-            Arrays.fill(buffer, (byte)random.nextInt(Byte.MAX_VALUE));
-          } else {
-            random.nextBytes(buffer);
-          }
-          long remains = size - written;
-          int length = (remains<=buffer.length) ? (int)remains : buffer.length;
-          out.write(buffer, 0, length);
-          written += length;
-          reporter.setStatus("writing "+name+"@"+written+"/"+size);
-        }
-      } finally {
-        out.close();
-      }
-      // rename to final location
-      fs.rename(tempFile, new Path(DATA_DIR, name));
-
-      collector.collect(new Text("bytes"), new LongWritable(written));
-
-      reporter.setStatus("wrote " + name);
-    }
-    
-    public void close() {
-    }
-    
-  }
-
-  public static void writeTest(FileSystem fs, boolean fastCheck)
-    throws Exception {
-
-    fs.delete(DATA_DIR, true);
-    fs.delete(WRITE_DIR, true);
-    
-    JobConf job = new JobConf(conf, TestFileSystem.class);
-    job.setBoolean("fs.test.fastCheck", fastCheck);
-
-    FileInputFormat.setInputPaths(job, CONTROL_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(WriteMapper.class);
-    job.setReducerClass(LongSumReducer.class);
-
-    FileOutputFormat.setOutputPath(job, WRITE_DIR);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(LongWritable.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-  public static class ReadMapper extends Configured
-      implements Mapper<Text, LongWritable, Text, LongWritable> {
-    
-    private Random random = new Random();
-    private byte[] buffer = new byte[BUFFER_SIZE];
-    private byte[] check  = new byte[BUFFER_SIZE];
-    private FileSystem fs;
-    private boolean fastCheck;
-
-    {
-      try {
-        fs = FileSystem.get(conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    public ReadMapper() { super(null); }
-    
-    public ReadMapper(Configuration conf) { super(conf); }
-
-    public void configure(JobConf job) {
-      setConf(job);
-      fastCheck = job.getBoolean("fs.test.fastCheck", false);
-    }
-
-    public void map(Text key, LongWritable value,
-                    OutputCollector<Text, LongWritable> collector,
-                    Reporter reporter)
-      throws IOException {
-      
-      String name = key.toString();
-      long size = value.get();
-      long seed = Long.parseLong(name);
-
-      random.setSeed(seed);
-      reporter.setStatus("opening " + name);
-
-      DataInputStream in =
-        new DataInputStream(fs.open(new Path(DATA_DIR, name)));
-
-      long read = 0;
-      try {
-        while (read < size) {
-          long remains = size - read;
-          int n = (remains<=buffer.length) ? (int)remains : buffer.length;
-          in.readFully(buffer, 0, n);
-          read += n;
-          if (fastCheck) {
-            Arrays.fill(check, (byte)random.nextInt(Byte.MAX_VALUE));
-          } else {
-            random.nextBytes(check);
-          }
-          if (n != buffer.length) {
-            Arrays.fill(buffer, n, buffer.length, (byte)0);
-            Arrays.fill(check, n, check.length, (byte)0);
-          }
-          assertTrue(Arrays.equals(buffer, check));
-
-          reporter.setStatus("reading "+name+"@"+read+"/"+size);
-
-        }
-      } finally {
-        in.close();
-      }
-
-      collector.collect(new Text("bytes"), new LongWritable(read));
-
-      reporter.setStatus("read " + name);
-    }
-    
-    public void close() {
-    }
-    
-  }
-
-  public static void readTest(FileSystem fs, boolean fastCheck)
-    throws Exception {
-
-    fs.delete(READ_DIR, true);
-
-    JobConf job = new JobConf(conf, TestFileSystem.class);
-    job.setBoolean("fs.test.fastCheck", fastCheck);
-
-
-    FileInputFormat.setInputPaths(job, CONTROL_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(ReadMapper.class);
-    job.setReducerClass(LongSumReducer.class);
-
-    FileOutputFormat.setOutputPath(job, READ_DIR);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(LongWritable.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-
-  public static class SeekMapper<K> extends Configured
-    implements Mapper<Text, LongWritable, K, LongWritable> {
-    
-    private Random random = new Random();
-    private byte[] check  = new byte[BUFFER_SIZE];
-    private FileSystem fs;
-    private boolean fastCheck;
-
-    {
-      try {
-        fs = FileSystem.get(conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    public SeekMapper() { super(null); }
-    
-    public SeekMapper(Configuration conf) { super(conf); }
-
-    public void configure(JobConf job) {
-      setConf(job);
-      fastCheck = job.getBoolean("fs.test.fastCheck", false);
-    }
-
-    public void map(Text key, LongWritable value,
-                    OutputCollector<K, LongWritable> collector,
-                    Reporter reporter)
-      throws IOException {
-      String name = key.toString();
-      long size = value.get();
-      long seed = Long.parseLong(name);
-
-      if (size == 0) return;
-
-      reporter.setStatus("opening " + name);
-
-      FSDataInputStream in = fs.open(new Path(DATA_DIR, name));
-        
-      try {
-        for (int i = 0; i < SEEKS_PER_FILE; i++) {
-          // generate a random position
-          long position = Math.abs(random.nextLong()) % size;
-          
-          // seek file to that position
-          reporter.setStatus("seeking " + name);
-          in.seek(position);
-          byte b = in.readByte();
-          
-          // check that byte matches
-          byte checkByte = 0;
-          // advance random state to that position
-          random.setSeed(seed);
-          for (int p = 0; p <= position; p+= check.length) {
-            reporter.setStatus("generating data for " + name);
-            if (fastCheck) {
-              checkByte = (byte)random.nextInt(Byte.MAX_VALUE);
-            } else {
-              random.nextBytes(check);
-              checkByte = check[(int)(position % check.length)];
-            }
-          }
-          assertEquals(b, checkByte);
-        }
-      } finally {
-        in.close();
-      }
-    }
-    
-    public void close() {
-    }
-    
-  }
-
-  public static void seekTest(FileSystem fs, boolean fastCheck)
-    throws Exception {
-
-    fs.delete(READ_DIR, true);
-
-    JobConf job = new JobConf(conf, TestFileSystem.class);
-    job.setBoolean("fs.test.fastCheck", fastCheck);
-
-    FileInputFormat.setInputPaths(job,CONTROL_DIR);
-    job.setInputFormat(SequenceFileInputFormat.class);
-
-    job.setMapperClass(SeekMapper.class);
-    job.setReducerClass(LongSumReducer.class);
-
-    FileOutputFormat.setOutputPath(job, READ_DIR);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(LongWritable.class);
-    job.setNumReduceTasks(1);
-    JobClient.runJob(job);
-  }
-
-
-  public static void main(String[] args) throws Exception {
-    int megaBytes = 10;
-    int files = 100;
-    boolean noRead = false;
-    boolean noWrite = false;
-    boolean noSeek = false;
-    boolean fastCheck = false;
-    long seed = new Random().nextLong();
-
-    String usage = "Usage: TestFileSystem -files N -megaBytes M [-noread] [-nowrite] [-noseek] [-fastcheck]";
-    
-    if (args.length == 0) {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    for (int i = 0; i < args.length; i++) {       // parse command line
-      if (args[i].equals("-files")) {
-        files = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-megaBytes")) {
-        megaBytes = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-noread")) {
-        noRead = true;
-      } else if (args[i].equals("-nowrite")) {
-        noWrite = true;
-      } else if (args[i].equals("-noseek")) {
-        noSeek = true;
-      } else if (args[i].equals("-fastcheck")) {
-        fastCheck = true;
-      }
-    }
-
-    LOG.info("seed = "+seed);
-    LOG.info("files = " + files);
-    LOG.info("megaBytes = " + megaBytes);
-  
-    FileSystem fs = FileSystem.get(conf);
-
-    if (!noWrite) {
-      createControlFile(fs, megaBytes*MEGA, files, seed);
-      writeTest(fs, fastCheck);
-    }
-    if (!noRead) {
-      readTest(fs, fastCheck);
-    }
-    if (!noSeek) {
-      seekTest(fs, fastCheck);
-    }
-  }
-
-  static Configuration createConf4Testing(String username) throws Exception {
-    Configuration conf = new Configuration();
-    UnixUserGroupInformation.saveToConf(conf,
-        UnixUserGroupInformation.UGI_PROPERTY_NAME,
-        new UnixUserGroupInformation(username, new String[]{"group"}));
-    return conf;    
-  }
-
-  public void testFsCache() throws Exception {
-    {
-      long now = System.currentTimeMillis();
-      Configuration[] conf = {new Configuration(),
-          createConf4Testing("foo" + now), createConf4Testing("bar" + now)};
-      FileSystem[] fs = new FileSystem[conf.length];
-  
-      for(int i = 0; i < conf.length; i++) {
-        fs[i] = FileSystem.get(conf[i]);
-        assertEquals(fs[i], FileSystem.get(conf[i]));
-        for(int j = 0; j < i; j++) {
-          assertFalse(fs[j] == fs[i]);
-        }
-      }
-      FileSystem.closeAll();
-    }
-    
-    {
-      try {
-        runTestCache(NameNode.DEFAULT_PORT);
-      } catch(java.net.BindException be) {
-        LOG.warn("Cannot test NameNode.DEFAULT_PORT (="
-            + NameNode.DEFAULT_PORT + ")", be);
-      }
-
-      runTestCache(0);
-    }
-  }
-  
-  static void runTestCache(int port) throws Exception {
-    Configuration conf = new Configuration();
-    MiniDFSCluster cluster = null;
-    try {
-      cluster = new MiniDFSCluster(port, conf, 2, true, true, null, null);
-      URI uri = cluster.getFileSystem().getUri();
-      LOG.info("uri=" + uri);
-
-      {
-        FileSystem fs = FileSystem.get(uri, new Configuration());
-        checkPath(cluster, fs);
-        for(int i = 0; i < 100; i++) {
-          assertTrue(fs == FileSystem.get(uri, new Configuration()));
-        }
-      }
-      
-      if (port == NameNode.DEFAULT_PORT) {
-        //test explicit default port
-        URI uri2 = new URI(uri.getScheme(), uri.getUserInfo(),
-            uri.getHost(), NameNode.DEFAULT_PORT, uri.getPath(),
-            uri.getQuery(), uri.getFragment());  
-        LOG.info("uri2=" + uri2);
-        FileSystem fs = FileSystem.get(uri2, conf);
-        checkPath(cluster, fs);
-        for(int i = 0; i < 100; i++) {
-          assertTrue(fs == FileSystem.get(uri2, new Configuration()));
-        }
-      }
-    } finally {
-      if (cluster != null) cluster.shutdown(); 
-    }
-  }
-  
-  static void checkPath(MiniDFSCluster cluster, FileSystem fileSys) throws IOException {
-    InetSocketAddress add = cluster.getNameNode().getNameNodeAddress();
-    // Test upper/lower case
-    fileSys.checkPath(new Path("hdfs://" + add.getHostName().toUpperCase() + ":" + add.getPort()));
-  }
-
-  public void testFsClose() throws Exception {
-    {
-      Configuration conf = new Configuration();
-      new Path("file:///").getFileSystem(conf);
-      UnixUserGroupInformation.login(conf, true);
-      FileSystem.closeAll();
-    }
-
-    {
-      Configuration conf = new Configuration();
-      new Path("hftp://localhost:12345/").getFileSystem(conf);
-      UnixUserGroupInformation.login(conf, true);
-      FileSystem.closeAll();
-    }
-
-    {
-      Configuration conf = new Configuration();
-      FileSystem fs = new Path("hftp://localhost:12345/").getFileSystem(conf);
-      UnixUserGroupInformation.login(fs.getConf(), true);
-      FileSystem.closeAll();
-    }
-  }
-
-  public void testFsShutdownHook() throws Exception {
-    final Set<FileSystem> closed = Collections.synchronizedSet(new HashSet<FileSystem>());
-    Configuration conf = new Configuration();
-    Configuration confNoAuto = new Configuration();
-
-    conf.setClass("fs.test.impl", TestShutdownFileSystem.class, FileSystem.class);
-    confNoAuto.setClass("fs.test.impl", TestShutdownFileSystem.class, FileSystem.class);
-    confNoAuto.setBoolean("fs.automatic.close", false);
-
-    TestShutdownFileSystem fsWithAuto =
-      (TestShutdownFileSystem)(new Path("test://a/").getFileSystem(conf));
-    TestShutdownFileSystem fsWithoutAuto =
-      (TestShutdownFileSystem)(new Path("test://b/").getFileSystem(confNoAuto));
-
-    fsWithAuto.setClosedSet(closed);
-    fsWithoutAuto.setClosedSet(closed);
-
-    // Different URIs should result in different FS instances
-    assertNotSame(fsWithAuto, fsWithoutAuto);
-
-    FileSystem.CACHE.closeAll(true);
-    assertEquals(1, closed.size());
-    assertTrue(closed.contains(fsWithAuto));
-
-    closed.clear();
-
-    FileSystem.closeAll();
-    assertEquals(1, closed.size());
-    assertTrue(closed.contains(fsWithoutAuto));
-  }
-
-
-  public void testCacheKeysAreCaseInsensitive()
-    throws Exception
-  {
-    Configuration conf = new Configuration();
-    
-    // check basic equality
-    FileSystem.Cache.Key lowercaseCachekey1 = new FileSystem.Cache.Key(new URI("hftp://localhost:12345/"), conf);
-    FileSystem.Cache.Key lowercaseCachekey2 = new FileSystem.Cache.Key(new URI("hftp://localhost:12345/"), conf);
-    assertEquals( lowercaseCachekey1, lowercaseCachekey2 );
-
-    // check insensitive equality    
-    FileSystem.Cache.Key uppercaseCachekey = new FileSystem.Cache.Key(new URI("HFTP://Localhost:12345/"), conf);
-    assertEquals( lowercaseCachekey2, uppercaseCachekey );
-
-    // check behaviour with collections
-    List<FileSystem.Cache.Key> list = new ArrayList<FileSystem.Cache.Key>();
-    list.add(uppercaseCachekey);
-    assertTrue(list.contains(uppercaseCachekey));
-    assertTrue(list.contains(lowercaseCachekey2));
-
-    Set<FileSystem.Cache.Key> set = new HashSet<FileSystem.Cache.Key>();
-    set.add(uppercaseCachekey);
-    assertTrue(set.contains(uppercaseCachekey));
-    assertTrue(set.contains(lowercaseCachekey2));
-
-    Map<FileSystem.Cache.Key, String> map = new HashMap<FileSystem.Cache.Key, String>();
-    map.put(uppercaseCachekey, "");
-    assertTrue(map.containsKey(uppercaseCachekey));
-    assertTrue(map.containsKey(lowercaseCachekey2));    
-
-  }
-
-  public static void testFsUniqueness(long megaBytes, int numFiles, long seed)
-    throws Exception {
-
-    // multiple invocations of FileSystem.get return the same object.
-    FileSystem fs1 = FileSystem.get(conf);
-    FileSystem fs2 = FileSystem.get(conf);
-    assertTrue(fs1 == fs2);
-
-    // multiple invocations of FileSystem.newInstance return different objects
-    fs1 = FileSystem.newInstance(conf);
-    fs2 = FileSystem.newInstance(conf);
-    assertTrue(fs1 != fs2 && !fs1.equals(fs2));
-    fs1.close();
-    fs2.close();
-  }
-
-  public static class TestShutdownFileSystem extends RawLocalFileSystem {
-    private Set<FileSystem> closedSet;
-
-    public void setClosedSet(Set<FileSystem> closedSet) {
-      this.closedSet = closedSet;
-    }
-    public void close() throws IOException {
-      if (closedSet != null) {
-        closedSet.add(this);
-      }
-      super.close();
-    }
-  }
-}

+ 0 - 146
src/test/hdfs-with-mr/org/apache/hadoop/fs/TestJHLA.java

@@ -1,146 +0,0 @@
-/**
- * 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.fs;
-
-import java.io.BufferedWriter;
-import java.io.FileOutputStream;
-import java.io.OutputStreamWriter;
-import java.io.File;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.After;
-import org.junit.Before;
-
-/**
- * Test Job History Log Analyzer.
- *
- * @see JHLogAnalyzer
- */
-public class TestJHLA extends TestCase {
-  private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class);
-  private String historyLog = System.getProperty("test.build.data", 
-                                  "build/test/data") + "/history/test.log";
-
-  @Before
-  public void setUp() throws Exception {
-    File logFile = new File(historyLog);
-    if(!logFile.getParentFile().exists())
-      if(!logFile.getParentFile().mkdirs())
-        LOG.error("Cannot create dirs for history log file: " + historyLog);
-    if(!logFile.createNewFile())
-      LOG.error("Cannot create history log file: " + historyLog);
-    BufferedWriter writer = new BufferedWriter(
-        new OutputStreamWriter(new FileOutputStream(historyLog)));
-    writer.write("$!!FILE=file1.log!!"); writer.newLine();
-    writer.write("Meta VERSION=\"1\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0004\" JOBNAME=\"streamjob21364.jar\" USER=\"hadoop\" SUBMIT_TIME=\"1237962008012\" JOBCONF=\"hdfs:///job_200903250600_0004/job.xml\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0004\" JOB_PRIORITY=\"NORMAL\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0004\" LAUNCH_TIME=\"1237962008712\" TOTAL_MAPS=\"2\" TOTAL_REDUCES=\"0\" JOB_STATUS=\"PREP\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000003\" TASK_TYPE=\"SETUP\" START_TIME=\"1237962008736\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0004_m_000003\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000003_0\" START_TIME=\"1237962010929\" TRACKER_NAME=\"tracker_50445\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0004_m_000003\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000003_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962012459\" HOSTNAME=\"host.com\" STATE_STRING=\"setup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000003\" TASK_TYPE=\"SETUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962023824\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0004\" JOB_STATUS=\"RUNNING\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000000\" TASK_TYPE=\"MAP\" START_TIME=\"1237962024049\" SPLITS=\"host1.com,host2.com,host3.com\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000001\" TASK_TYPE=\"MAP\" START_TIME=\"1237962024065\" SPLITS=\"host1.com,host2.com,host3.com\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0004_m_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000000_0\" START_TIME=\"1237962026157\" TRACKER_NAME=\"tracker_50524\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0004_m_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000000_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962041307\" HOSTNAME=\"host.com\" STATE_STRING=\"Records R/W=2681/1\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(56630)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(28327)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(2681)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(28327)][(MAP_OUTPUT_RECORDS)(Map output records)(2681)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000000\" TASK_TYPE=\"MAP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962054138\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(56630)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(28327)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(2681)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(28327)][(MAP_OUTPUT_RECORDS)(Map output records)(2681)]}\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0004_m_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000001_0\" START_TIME=\"1237962026077\" TRACKER_NAME=\"tracker_50162\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0004_m_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000001_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962041030\" HOSTNAME=\"host.com\" STATE_STRING=\"Records R/W=2634/1\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(28316)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(28303)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(2634)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(28303)][(MAP_OUTPUT_RECORDS)(Map output records)(2634)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000001\" TASK_TYPE=\"MAP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962054187\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(28316)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(28303)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(2634)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(28303)][(MAP_OUTPUT_RECORDS)(Map output records)(2634)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000002\" TASK_TYPE=\"CLEANUP\" START_TIME=\"1237962054187\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0004_m_000002\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000002_0\" START_TIME=\"1237962055578\" TRACKER_NAME=\"tracker_50162\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0004_m_000002\" TASK_ATTEMPT_ID=\"attempt_200903250600_0004_m_000002_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962056782\" HOSTNAME=\"host.com\" STATE_STRING=\"cleanup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0004_m_000002\" TASK_TYPE=\"CLEANUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237962069193\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0004\" FINISH_TIME=\"1237962069193\" JOB_STATUS=\"SUCCESS\" FINISHED_MAPS=\"2\" FINISHED_REDUCES=\"0\" FAILED_MAPS=\"0\" FAILED_REDUCES=\"0\" COUNTERS=\"{(org.apache.hadoop.mapred.JobInProgress$Counter)(Job Counters )[(TOTAL_LAUNCHED_MAPS)(Launched map tasks)(2)]}{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(84946)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(56630)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(5315)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(56630)][(MAP_OUTPUT_RECORDS)(Map output records)(5315)]}\" ."); writer.newLine();
-    writer.write("$!!FILE=file2.log!!"); writer.newLine();
-    writer.write("Meta VERSION=\"1\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0023\" JOBNAME=\"TestJob\" USER=\"hadoop2\" SUBMIT_TIME=\"1237964779799\" JOBCONF=\"hdfs:///job_200903250600_0023/job.xml\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0023\" JOB_PRIORITY=\"NORMAL\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0023\" LAUNCH_TIME=\"1237964780928\" TOTAL_MAPS=\"2\" TOTAL_REDUCES=\"0\" JOB_STATUS=\"PREP\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_r_000001\" TASK_TYPE=\"SETUP\" START_TIME=\"1237964780940\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("ReduceAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0023_r_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_r_000001_0\" START_TIME=\"1237964720322\" TRACKER_NAME=\"tracker_3065\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("ReduceAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0023_r_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_r_000001_0\" TASK_STATUS=\"SUCCESS\" SHUFFLE_FINISHED=\"1237964722118\" SORT_FINISHED=\"1237964722118\" FINISH_TIME=\"1237964722118\" HOSTNAME=\"host.com\" STATE_STRING=\"setup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(0)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_r_000001\" TASK_TYPE=\"SETUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964796054\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(0)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0023\" JOB_STATUS=\"RUNNING\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_m_000000\" TASK_TYPE=\"MAP\" START_TIME=\"1237964796176\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_m_000001\" TASK_TYPE=\"MAP\" START_TIME=\"1237964796176\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0023_m_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_m_000000_0\" START_TIME=\"1237964809765\" TRACKER_NAME=\"tracker_50459\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0023_m_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_m_000000_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964911772\" HOSTNAME=\"host.com\" STATE_STRING=\"\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(500000000)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(5000000)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(5000000)][(MAP_OUTPUT_RECORDS)(Map output records)(5000000)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_m_000000\" TASK_TYPE=\"MAP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964916534\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(500000000)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(5000000)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(5000000)][(MAP_OUTPUT_RECORDS)(Map output records)(5000000)]}\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0023_m_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_m_000001_0\" START_TIME=\"1237964798169\" TRACKER_NAME=\"tracker_1524\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0023_m_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_m_000001_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964962960\" HOSTNAME=\"host.com\" STATE_STRING=\"\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(500000000)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(5000000)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(5000000)][(MAP_OUTPUT_RECORDS)(Map output records)(5000000)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_m_000001\" TASK_TYPE=\"MAP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964976870\" COUNTERS=\"{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(500000000)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(5000000)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(5000000)][(MAP_OUTPUT_RECORDS)(Map output records)(5000000)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_r_000000\" TASK_TYPE=\"CLEANUP\" START_TIME=\"1237964976871\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("ReduceAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0023_r_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_r_000000_0\" START_TIME=\"1237964977208\" TRACKER_NAME=\"tracker_1524\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("ReduceAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0023_r_000000\" TASK_ATTEMPT_ID=\"attempt_200903250600_0023_r_000000_0\" TASK_STATUS=\"SUCCESS\" SHUFFLE_FINISHED=\"1237964979031\" SORT_FINISHED=\"1237964979031\" FINISH_TIME=\"1237964979032\" HOSTNAME=\"host.com\" STATE_STRING=\"cleanup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(0)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0023_r_000000\" TASK_TYPE=\"CLEANUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237964991879\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(0)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0023\" FINISH_TIME=\"1237964991879\" JOB_STATUS=\"SUCCESS\" FINISHED_MAPS=\"2\" FINISHED_REDUCES=\"0\" FAILED_MAPS=\"0\" FAILED_REDUCES=\"0\" COUNTERS=\"{(org.apache.hadoop.mapred.JobInProgress$Counter)(Job Counters )[(TOTAL_LAUNCHED_MAPS)(Launched map tasks)(2)]}{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(1000000000)]}{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(MAP_INPUT_RECORDS)(Map input records)(10000000)][(SPILLED_RECORDS)(Spilled Records)(0)][(MAP_INPUT_BYTES)(Map input bytes)(10000000)][(MAP_OUTPUT_RECORDS)(Map output records)(10000000)]}\" ."); writer.newLine();
-    writer.write("$!!FILE=file3.log!!"); writer.newLine();
-    writer.write("Meta VERSION=\"1\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0034\" JOBNAME=\"TestJob\" USER=\"hadoop3\" SUBMIT_TIME=\"1237966370007\" JOBCONF=\"hdfs:///job_200903250600_0034/job.xml\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0034\" JOB_PRIORITY=\"NORMAL\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0034\" LAUNCH_TIME=\"1237966371076\" TOTAL_MAPS=\"2\" TOTAL_REDUCES=\"0\" JOB_STATUS=\"PREP\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000003\" TASK_TYPE=\"SETUP\" START_TIME=\"1237966371093\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0034_m_000003\" TASK_ATTEMPT_ID=\"attempt_200903250600_0034_m_000003_0\" START_TIME=\"1237966371524\" TRACKER_NAME=\"tracker_50118\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"SETUP\" TASKID=\"task_200903250600_0034_m_000003\" TASK_ATTEMPT_ID=\"attempt_200903250600_0034_m_000003_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237966373174\" HOSTNAME=\"host.com\" STATE_STRING=\"setup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000003\" TASK_TYPE=\"SETUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237966386098\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0034\" JOB_STATUS=\"RUNNING\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000000\" TASK_TYPE=\"MAP\" START_TIME=\"1237966386111\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000001\" TASK_TYPE=\"MAP\" START_TIME=\"1237966386124\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"MAP\" TASKID=\"task_200903250600_0034_m_000001\" TASK_ATTEMPT_ID=\"attempt_200903250600_0034_m_000001_0\" TASK_STATUS=\"FAILED\" FINISH_TIME=\"1237967174546\" HOSTNAME=\"host.com\" ERROR=\"java.io.IOException: Task process exit with nonzero status of 15."); writer.newLine();
-    writer.write("  at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:424)"); writer.newLine();
-    writer.write(",java.io.IOException: Task process exit with nonzero status of 15."); writer.newLine();
-    writer.write("  at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:424)"); writer.newLine();
-    writer.write("\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000002\" TASK_TYPE=\"CLEANUP\" START_TIME=\"1237967170815\" SPLITS=\"\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0034_m_000002\" TASK_ATTEMPT_ID=\"attempt_200903250600_0034_m_000002_0\" START_TIME=\"1237967168653\" TRACKER_NAME=\"tracker_3105\" HTTP_PORT=\"50060\" ."); writer.newLine();
-    writer.write("MapAttempt TASK_TYPE=\"CLEANUP\" TASKID=\"task_200903250600_0034_m_000002\" TASK_ATTEMPT_ID=\"attempt_200903250600_0034_m_000002_0\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237967171301\" HOSTNAME=\"host.com\" STATE_STRING=\"cleanup\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Task TASKID=\"task_200903250600_0034_m_000002\" TASK_TYPE=\"CLEANUP\" TASK_STATUS=\"SUCCESS\" FINISH_TIME=\"1237967185818\" COUNTERS=\"{(org.apache.hadoop.mapred.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}\" ."); writer.newLine();
-    writer.write("Job JOBID=\"job_200903250600_0034\" FINISH_TIME=\"1237967185818\" JOB_STATUS=\"KILLED\" FINISHED_MAPS=\"0\" FINISHED_REDUCES=\"0\" ."); writer.newLine();
-    writer.close();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    File logFile = new File(historyLog);
-    if(!logFile.delete())
-      LOG.error("Cannot delete history log file: " + historyLog);
-    if(!logFile.getParentFile().delete())
-      LOG.error("Cannot delete history log dir: " + historyLog);
-  }
-
-  /**
-   * Run log analyzer in test mode for file test.log.
-   */
-  public void testJHLA() {
-    String[] args = {"-test", historyLog, "-jobDelimiter", ".!!FILE=.*!!"};
-    JHLogAnalyzer.main(args);
-    args = new String[]{"-test", historyLog, "-jobDelimiter", ".!!FILE=.*!!",
-                        "-usersIncluded", "hadoop,hadoop2"};
-    JHLogAnalyzer.main(args);
-    args = new String[]{"-test", historyLog, "-jobDelimiter", ".!!FILE=.*!!",
-        "-usersExcluded", "hadoop,hadoop3"};
-    JHLogAnalyzer.main(args);
-  }
-}

+ 0 - 975
src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBench.java

@@ -1,975 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.util.Date;
-import java.io.DataInputStream;
-import java.io.FileOutputStream;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
-import java.io.File;
-import java.io.BufferedReader;
-import java.util.StringTokenizer;
-import java.net.InetAddress;
-import java.text.SimpleDateFormat;
-import java.util.Iterator;
-
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.Log;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile;
-
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reducer;
-
-/**
- * This program executes a specified operation that applies load to 
- * the NameNode.
- * 
- * When run simultaneously on multiple nodes, this program functions 
- * as a stress-test and benchmark for namenode, especially when 
- * the number of bytes written to each file is small.
- * 
- * Valid operations are:
- *   create_write
- *   open_read
- *   rename
- *   delete
- * 
- * NOTE: The open_read, rename and delete operations assume that the files
- *       they operate on are already available. The create_write operation 
- *       must be run before running the other operations.
- */
-
-public class NNBench {
-  private static final Log LOG = LogFactory.getLog(
-          "org.apache.hadoop.hdfs.NNBench");
-  
-  protected static String CONTROL_DIR_NAME = "control";
-  protected static String OUTPUT_DIR_NAME = "output";
-  protected static String DATA_DIR_NAME = "data";
-  protected static final String DEFAULT_RES_FILE_NAME = "NNBench_results.log";
-  protected static final String NNBENCH_VERSION = "NameNode Benchmark 0.4";
-  
-  public static String operation = "none";
-  public static long numberOfMaps = 1l; // default is 1
-  public static long numberOfReduces = 1l; // default is 1
-  public static long startTime = 
-          System.currentTimeMillis() + (120 * 1000); // default is 'now' + 2min
-  public static long blockSize = 1l; // default is 1
-  public static int bytesToWrite = 0; // default is 0
-  public static long bytesPerChecksum = 1l; // default is 1
-  public static long numberOfFiles = 1l; // default is 1
-  public static short replicationFactorPerFile = 1; // default is 1
-  public static String baseDir = "/benchmarks/NNBench";  // default
-  public static boolean readFileAfterOpen = false; // default is to not read
-  
-  // Supported operations
-  private static final String OP_CREATE_WRITE = "create_write";
-  private static final String OP_OPEN_READ = "open_read";
-  private static final String OP_RENAME = "rename";
-  private static final String OP_DELETE = "delete";
-  
-  // To display in the format that matches the NN and DN log format
-  // Example: 2007-10-26 00:01:19,853
-  static SimpleDateFormat sdf = 
-          new SimpleDateFormat("yyyy-MM-dd' 'HH:mm:ss','S");
-
-  private static Configuration config = new Configuration();
-  
-  /**
-   * Clean up the files before a test run
-   * 
-   * @throws IOException on error
-   */
-  private static void cleanupBeforeTestrun() throws IOException {
-    FileSystem tempFS = FileSystem.get(config);
-    
-    // Delete the data directory only if it is the create/write operation
-    if (operation.equals(OP_CREATE_WRITE)) {
-      LOG.info("Deleting data directory");
-      tempFS.delete(new Path(baseDir, DATA_DIR_NAME), true);
-    }
-    tempFS.delete(new Path(baseDir, CONTROL_DIR_NAME), true);
-    tempFS.delete(new Path(baseDir, OUTPUT_DIR_NAME), true);
-  }
-  
-  /**
-   * Create control files before a test run.
-   * Number of files created is equal to the number of maps specified
-   * 
-   * @throws IOException on error
-   */
-  private static void createControlFiles() throws IOException {
-    FileSystem tempFS = FileSystem.get(config);
-    LOG.info("Creating " + numberOfMaps + " control files");
-
-    for (int i = 0; i < numberOfMaps; i++) {
-      String strFileName = "NNBench_Controlfile_" + i;
-      Path filePath = new Path(new Path(baseDir, CONTROL_DIR_NAME),
-              strFileName);
-
-      SequenceFile.Writer writer = null;
-      try {
-        writer = SequenceFile.createWriter(tempFS, config, filePath, Text.class, 
-                LongWritable.class, CompressionType.NONE);
-        writer.append(new Text(strFileName), new LongWritable(0l));
-      } finally {
-        if (writer != null) {
-          writer.close();
-        }
-      }
-    }
-  }
-  /**
-   * Display version
-   */
-  private static void displayVersion() {
-    System.out.println(NNBENCH_VERSION);
-  }
-  
-  /**
-   * Display usage
-   */
-  private static void displayUsage() {
-    String usage =
-      "Usage: nnbench <options>\n" +
-      "Options:\n" +
-      "\t-operation <Available operations are " + OP_CREATE_WRITE + " " +
-      OP_OPEN_READ + " " + OP_RENAME + " " + OP_DELETE + ". " +
-      "This option is mandatory>\n" +
-      "\t * NOTE: The open_read, rename and delete operations assume " +
-      "that the files they operate on, are already available. " +
-      "The create_write operation must be run before running the " +
-      "other operations.\n" +
-      "\t-maps <number of maps. default is 1. This is not mandatory>\n" +
-      "\t-reduces <number of reduces. default is 1. This is not mandatory>\n" +
-      "\t-startTime <time to start, given in seconds from the epoch. " +
-      "Make sure this is far enough into the future, so all maps " +
-      "(operations) will start at the same time>. " +
-      "default is launch time + 2 mins. This is not mandatory \n" +
-      "\t-blockSize <Block size in bytes. default is 1. " + 
-      "This is not mandatory>\n" +
-      "\t-bytesToWrite <Bytes to write. default is 0. " + 
-      "This is not mandatory>\n" +
-      "\t-bytesPerChecksum <Bytes per checksum for the files. default is 1. " + 
-      "This is not mandatory>\n" +
-      "\t-numberOfFiles <number of files to create. default is 1. " +
-      "This is not mandatory>\n" +
-      "\t-replicationFactorPerFile <Replication factor for the files." +
-        " default is 1. This is not mandatory>\n" +
-      "\t-baseDir <base DFS path. default is /becnhmarks/NNBench. " +
-      "This is not mandatory>\n" +
-      "\t-readFileAfterOpen <true or false. if true, it reads the file and " +
-      "reports the average time to read. This is valid with the open_read " +
-      "operation. default is false. This is not mandatory>\n" +
-      "\t-help: Display the help statement\n";
-      
-    
-    System.out.println(usage);
-  }
-
-  /**
-   * check for arguments and fail if the values are not specified
-   * @param index  positional number of an argument in the list of command
-   *   line's arguments
-   * @param length total number of arguments
-   */
-  public static void checkArgs(final int index, final int length) {
-    if (index == length) {
-      displayUsage();
-      System.exit(-1);
-    }
-  }
-  
-  /**
-   * Parse input arguments
-   *
-   * @param args array of command line's parameters to be parsed
-   */
-  public static void parseInputs(final String[] args) {
-    // If there are no command line arguments, exit
-    if (args.length == 0) {
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // Parse command line args
-    for (int i = 0; i < args.length; i++) {
-      if (args[i].equals("-operation")) {
-        operation = args[++i];
-      } else if (args[i].equals("-maps")) {
-        checkArgs(i + 1, args.length);
-        numberOfMaps = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-reduces")) {
-        checkArgs(i + 1, args.length);
-        numberOfReduces = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-startTime")) {
-        checkArgs(i + 1, args.length);
-        startTime = Long.parseLong(args[++i]) * 1000;
-      } else if (args[i].equals("-blockSize")) {
-        checkArgs(i + 1, args.length);
-        blockSize = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-bytesToWrite")) {
-        checkArgs(i + 1, args.length);
-        bytesToWrite = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-bytesPerChecksum")) {
-        checkArgs(i + 1, args.length);
-        bytesPerChecksum = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-numberOfFiles")) {
-        checkArgs(i + 1, args.length);
-        numberOfFiles = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-replicationFactorPerFile")) {
-        checkArgs(i + 1, args.length);
-        replicationFactorPerFile = Short.parseShort(args[++i]);
-      } else if (args[i].equals("-baseDir")) {
-        checkArgs(i + 1, args.length);
-        baseDir = args[++i];
-      } else if (args[i].equals("-readFileAfterOpen")) {
-        checkArgs(i + 1, args.length);
-        readFileAfterOpen = Boolean.parseBoolean(args[++i]);
-      } else if (args[i].equals("-help")) {
-        displayUsage();
-        System.exit(-1);
-      }
-    }
-    
-    LOG.info("Test Inputs: ");
-    LOG.info("           Test Operation: " + operation);
-    LOG.info("               Start time: " + sdf.format(new Date(startTime)));
-    LOG.info("           Number of maps: " + numberOfMaps);
-    LOG.info("        Number of reduces: " + numberOfReduces);
-    LOG.info("               Block Size: " + blockSize);
-    LOG.info("           Bytes to write: " + bytesToWrite);
-    LOG.info("       Bytes per checksum: " + bytesPerChecksum);
-    LOG.info("          Number of files: " + numberOfFiles);
-    LOG.info("       Replication factor: " + replicationFactorPerFile);
-    LOG.info("                 Base dir: " + baseDir);
-    LOG.info("     Read file after open: " + readFileAfterOpen);
-    
-    // Set user-defined parameters, so the map method can access the values
-    config.set("test.nnbench.operation", operation);
-    config.setLong("test.nnbench.maps", numberOfMaps);
-    config.setLong("test.nnbench.reduces", numberOfReduces);
-    config.setLong("test.nnbench.starttime", startTime);
-    config.setLong("test.nnbench.blocksize", blockSize);
-    config.setInt("test.nnbench.bytestowrite", bytesToWrite);
-    config.setLong("test.nnbench.bytesperchecksum", bytesPerChecksum);
-    config.setLong("test.nnbench.numberoffiles", numberOfFiles);
-    config.setInt("test.nnbench.replicationfactor", 
-            (int) replicationFactorPerFile);
-    config.set("test.nnbench.basedir", baseDir);
-    config.setBoolean("test.nnbench.readFileAfterOpen", readFileAfterOpen);
-
-    config.set("test.nnbench.datadir.name", DATA_DIR_NAME);
-    config.set("test.nnbench.outputdir.name", OUTPUT_DIR_NAME);
-    config.set("test.nnbench.controldir.name", CONTROL_DIR_NAME);
-  }
-  
-  /**
-   * Analyze the results
-   * 
-   * @throws IOException on error
-   */
-  private static void analyzeResults() throws IOException {
-    final FileSystem fs = FileSystem.get(config);
-    Path reduceFile = new Path(new Path(baseDir, OUTPUT_DIR_NAME),
-            "part-00000");
-
-    DataInputStream in;
-    in = new DataInputStream(fs.open(reduceFile));
-
-    BufferedReader lines;
-    lines = new BufferedReader(new InputStreamReader(in));
-
-    long totalTimeAL1 = 0l;
-    long totalTimeAL2 = 0l;
-    long totalTimeTPmS = 0l;
-    long lateMaps = 0l;
-    long numOfExceptions = 0l;
-    long successfulFileOps = 0l;
-    
-    long mapStartTimeTPmS = 0l;
-    long mapEndTimeTPmS = 0l;
-    
-    String resultTPSLine1 = null;
-    String resultTPSLine2 = null;
-    String resultALLine1 = null;
-    String resultALLine2 = null;
-    
-    String line;
-    while((line = lines.readLine()) != null) {
-      StringTokenizer tokens = new StringTokenizer(line, " \t\n\r\f%;");
-      String attr = tokens.nextToken();
-      if (attr.endsWith(":totalTimeAL1")) {
-        totalTimeAL1 = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":totalTimeAL2")) {
-        totalTimeAL2 = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":totalTimeTPmS")) {
-        totalTimeTPmS = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":latemaps")) {
-        lateMaps = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":numOfExceptions")) {
-        numOfExceptions = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":successfulFileOps")) {
-        successfulFileOps = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":mapStartTimeTPmS")) {
-        mapStartTimeTPmS = Long.parseLong(tokens.nextToken());
-      } else if (attr.endsWith(":mapEndTimeTPmS")) {
-        mapEndTimeTPmS = Long.parseLong(tokens.nextToken());
-      }
-    }
-    
-    // Average latency is the average time to perform 'n' number of
-    // operations, n being the number of files
-    double avgLatency1 = (double) totalTimeAL1 / successfulFileOps;
-    double avgLatency2 = (double) totalTimeAL2 / successfulFileOps;
-    
-    // The time it takes for the longest running map is measured. Using that,
-    // cluster transactions per second is calculated. It includes time to 
-    // retry any of the failed operations
-    double longestMapTimeTPmS = (double) (mapEndTimeTPmS - mapStartTimeTPmS);
-    double totalTimeTPS = (longestMapTimeTPmS == 0) ?
-            (1000 * successfulFileOps) :
-            (double) (1000 * successfulFileOps) / longestMapTimeTPmS;
-            
-    // The time it takes to perform 'n' operations is calculated (in ms),
-    // n being the number of files. Using that time, the average execution 
-    // time is calculated. It includes time to retry any of the
-    // failed operations
-    double AverageExecutionTime = (totalTimeTPmS == 0) ?
-        (double) successfulFileOps : 
-        (double) totalTimeTPmS / successfulFileOps;
-            
-    if (operation.equals(OP_CREATE_WRITE)) {
-      // For create/write/close, it is treated as two transactions,
-      // since a file create from a client perspective involves create and close
-      resultTPSLine1 = "               TPS: Create/Write/Close: " + 
-        (int) (totalTimeTPS * 2);
-      resultTPSLine2 = "Avg exec time (ms): Create/Write/Close: " +
-        AverageExecutionTime;
-      resultALLine1 = "            Avg Lat (ms): Create/Write: " + avgLatency1;
-      resultALLine2 = "                   Avg Lat (ms): Close: " + avgLatency2;
-    } else if (operation.equals(OP_OPEN_READ)) {
-      resultTPSLine1 = "                        TPS: Open/Read: " + 
-        (int) totalTimeTPS;
-      resultTPSLine2 = "         Avg Exec time (ms): Open/Read: " + 
-        AverageExecutionTime;
-      resultALLine1 = "                    Avg Lat (ms): Open: " + avgLatency1;
-      if (readFileAfterOpen) {
-        resultALLine2 = "                  Avg Lat (ms): Read: " + avgLatency2;
-      }
-    } else if (operation.equals(OP_RENAME)) {
-      resultTPSLine1 = "                           TPS: Rename: " + 
-        (int) totalTimeTPS;
-      resultTPSLine2 = "            Avg Exec time (ms): Rename: " + 
-        AverageExecutionTime;
-      resultALLine1 = "                  Avg Lat (ms): Rename: " + avgLatency1;
-    } else if (operation.equals(OP_DELETE)) {
-      resultTPSLine1 = "                           TPS: Delete: " + 
-        (int) totalTimeTPS;
-      resultTPSLine2 = "            Avg Exec time (ms): Delete: " + 
-        AverageExecutionTime;
-      resultALLine1 = "                  Avg Lat (ms): Delete: " + avgLatency1;
-    }
-    
-    String resultLines[] = {
-    "-------------- NNBench -------------- : ",
-    "                               Version: " + NNBENCH_VERSION,
-    "                           Date & time: " + sdf.format(new Date(
-            System.currentTimeMillis())),
-    "",
-    "                        Test Operation: " + operation,
-    "                            Start time: " + 
-      sdf.format(new Date(startTime)),
-    "                           Maps to run: " + numberOfMaps,
-    "                        Reduces to run: " + numberOfReduces,
-    "                    Block Size (bytes): " + blockSize,
-    "                        Bytes to write: " + bytesToWrite,
-    "                    Bytes per checksum: " + bytesPerChecksum,
-    "                       Number of files: " + numberOfFiles,
-    "                    Replication factor: " + replicationFactorPerFile,
-    "            Successful file operations: " + successfulFileOps,
-    "",
-    "        # maps that missed the barrier: " + lateMaps,
-    "                          # exceptions: " + numOfExceptions,
-    "",
-    resultTPSLine1,
-    resultTPSLine2,
-    resultALLine1,
-    resultALLine2,
-    "",
-    "                 RAW DATA: AL Total #1: " + totalTimeAL1,
-    "                 RAW DATA: AL Total #2: " + totalTimeAL2,
-    "              RAW DATA: TPS Total (ms): " + totalTimeTPmS,
-    "       RAW DATA: Longest Map Time (ms): " + longestMapTimeTPmS,
-    "                   RAW DATA: Late maps: " + lateMaps,
-    "             RAW DATA: # of exceptions: " + numOfExceptions,
-    "" };
-
-    PrintStream res = new PrintStream(new FileOutputStream(
-            new File(DEFAULT_RES_FILE_NAME), true));
-    
-    // Write to a file and also dump to log
-    for(int i = 0; i < resultLines.length; i++) {
-      LOG.info(resultLines[i]);
-      res.println(resultLines[i]);
-    }
-  }
-  
-  /**
-   * Run the test
-   * 
-   * @throws IOException on error
-   */
-  public static void runTests() throws IOException {
-    config.setLong("io.bytes.per.checksum", bytesPerChecksum);
-    
-    JobConf job = new JobConf(config, NNBench.class);
-
-    job.setJobName("NNBench-" + operation);
-    FileInputFormat.setInputPaths(job, new Path(baseDir, CONTROL_DIR_NAME));
-    job.setInputFormat(SequenceFileInputFormat.class);
-    
-    // Explicitly set number of max map attempts to 1.
-    job.setMaxMapAttempts(1);
-    
-    // Explicitly turn off speculative execution
-    job.setSpeculativeExecution(false);
-
-    job.setMapperClass(NNBenchMapper.class);
-    job.setReducerClass(NNBenchReducer.class);
-
-    FileOutputFormat.setOutputPath(job, new Path(baseDir, OUTPUT_DIR_NAME));
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setNumReduceTasks((int) numberOfReduces);
-    JobClient.runJob(job);
-  }
-  
-  /**
-   * Validate the inputs
-   */
-  public static void validateInputs() {
-    // If it is not one of the four operations, then fail
-    if (!operation.equals(OP_CREATE_WRITE) &&
-            !operation.equals(OP_OPEN_READ) &&
-            !operation.equals(OP_RENAME) &&
-            !operation.equals(OP_DELETE)) {
-      System.err.println("Error: Unknown operation: " + operation);
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If number of maps is a negative number, then fail
-    // Hadoop allows the number of maps to be 0
-    if (numberOfMaps < 0) {
-      System.err.println("Error: Number of maps must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If number of reduces is a negative number or 0, then fail
-    if (numberOfReduces <= 0) {
-      System.err.println("Error: Number of reduces must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-
-    // If blocksize is a negative number or 0, then fail
-    if (blockSize <= 0) {
-      System.err.println("Error: Block size must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If bytes to write is a negative number, then fail
-    if (bytesToWrite < 0) {
-      System.err.println("Error: Bytes to write must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If bytes per checksum is a negative number, then fail
-    if (bytesPerChecksum < 0) {
-      System.err.println("Error: Bytes per checksum must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If number of files is a negative number, then fail
-    if (numberOfFiles < 0) {
-      System.err.println("Error: Number of files must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If replication factor is a negative number, then fail
-    if (replicationFactorPerFile < 0) {
-      System.err.println("Error: Replication factor must be a positive number");
-      displayUsage();
-      System.exit(-1);
-    }
-    
-    // If block size is not a multiple of bytesperchecksum, fail
-    if (blockSize % bytesPerChecksum != 0) {
-      System.err.println("Error: Block Size in bytes must be a multiple of " +
-              "bytes per checksum: ");
-      displayUsage();
-      System.exit(-1);
-    }
-  }
-  /**
-  * Main method for running the NNBench benchmarks
-  *
-  * @param args array of command line arguments
-  * @throws IOException indicates a problem with test startup
-  */
-  public static void main(String[] args) throws IOException {
-    // Display the application version string
-    displayVersion();
-
-    // Parse the inputs
-    parseInputs(args);
-    
-    // Validate inputs
-    validateInputs();
-    
-    // Clean up files before the test run
-    cleanupBeforeTestrun();
-    
-    // Create control files before test run
-    createControlFiles();
-
-    // Run the tests as a map reduce job
-    runTests();
-    
-    // Analyze results
-    analyzeResults();
-  }
-
-  
-  /**
-   * Mapper class
-   */
-  static class NNBenchMapper extends Configured
-          implements Mapper<Text, LongWritable, Text, Text> {
-    FileSystem filesystem = null;
-    private String hostName = null;
-
-    long numberOfFiles = 1l;
-    long blkSize = 1l;
-    short replFactor = 1;
-    int bytesToWrite = 0;
-    String baseDir = null;
-    String dataDirName = null;
-    String op = null;
-    boolean readFile = false;
-    final int MAX_OPERATION_EXCEPTIONS = 1000;
-    
-    // Data to collect from the operation
-    int numOfExceptions = 0;
-    long startTimeAL = 0l;
-    long totalTimeAL1 = 0l;
-    long totalTimeAL2 = 0l;
-    long successfulFileOps = 0l;
-    
-    /**
-     * Constructor
-     */
-    public NNBenchMapper() {
-    }
-    
-    /**
-     * Mapper base implementation
-     */
-    public void configure(JobConf conf) {
-      setConf(conf);
-      
-      try {
-        filesystem = FileSystem.get(conf);
-      } catch(Exception e) {
-        throw new RuntimeException("Cannot get file system.", e);
-      }
-      
-      try {
-        hostName = InetAddress.getLocalHost().getHostName();
-      } catch(Exception e) {
-        throw new RuntimeException("Error getting hostname", e);
-      }
-    }
-    
-    /**
-     * Mapper base implementation
-     */
-    public void close() throws IOException {
-    }
-
-    /**
-     * Returns when the current number of seconds from the epoch equals
-     * the command line argument given by <code>-startTime</code>.
-     * This allows multiple instances of this program, running on clock
-     * synchronized nodes, to start at roughly the same time.
-     * @return true if the method was able to sleep for <code>-startTime</code>
-     * without interruption; false otherwise
-     */
-    private boolean barrier() {
-      long startTime = getConf().getLong("test.nnbench.starttime", 0l);
-      long currentTime = System.currentTimeMillis();
-      long sleepTime = startTime - currentTime;
-      boolean retVal = false;
-      
-      // If the sleep time is greater than 0, then sleep and return
-      if (sleepTime > 0) {
-        LOG.info("Waiting in barrier for: " + sleepTime + " ms");
-      
-        try {
-          Thread.sleep(sleepTime);
-          retVal = true;
-        } catch (Exception e) {
-          retVal = false;
-        }
-      }
-      
-      return retVal;
-    }
-    
-    /**
-     * Map method
-     */ 
-    public void map(Text key, 
-            LongWritable value,
-            OutputCollector<Text, Text> output,
-            Reporter reporter) throws IOException {
-      Configuration conf = filesystem.getConf();
-      
-      numberOfFiles = conf.getLong("test.nnbench.numberoffiles", 1l);
-      blkSize = conf.getLong("test.nnbench.blocksize", 1l);
-      replFactor = (short) (conf.getInt("test.nnbench.replicationfactor", 1));
-      bytesToWrite = conf.getInt("test.nnbench.bytestowrite", 0);
-      baseDir = conf.get("test.nnbench.basedir");
-      dataDirName = conf.get("test.nnbench.datadir.name");
-      op = conf.get("test.nnbench.operation");
-      readFile = conf.getBoolean("test.nnbench.readFileAfterOpen", false);
-      
-      long totalTimeTPmS = 0l;
-      long startTimeTPmS = 0l;
-      long endTimeTPms = 0l;
-      
-      numOfExceptions = 0;
-      startTimeAL = 0l;
-      totalTimeAL1 = 0l;
-      totalTimeAL2 = 0l;
-      successfulFileOps = 0l;
-      
-      if (barrier()) {
-        if (op.equals(OP_CREATE_WRITE)) {
-          startTimeTPmS = System.currentTimeMillis();
-          doCreateWriteOp("file_" + hostName + "_", reporter);
-        } else if (op.equals(OP_OPEN_READ)) {
-          startTimeTPmS = System.currentTimeMillis();
-          doOpenReadOp("file_" + hostName + "_", reporter);
-        } else if (op.equals(OP_RENAME)) {
-          startTimeTPmS = System.currentTimeMillis();
-          doRenameOp("file_" + hostName + "_", reporter);
-        } else if (op.equals(OP_DELETE)) {
-          startTimeTPmS = System.currentTimeMillis();
-          doDeleteOp("file_" + hostName + "_", reporter);
-        }
-        
-        endTimeTPms = System.currentTimeMillis();
-        totalTimeTPmS = endTimeTPms - startTimeTPmS;
-      } else {
-        output.collect(new Text("l:latemaps"), new Text("1"));
-      }
-      
-      // collect after the map end time is measured
-      output.collect(new Text("l:totalTimeAL1"), 
-          new Text(String.valueOf(totalTimeAL1)));
-      output.collect(new Text("l:totalTimeAL2"), 
-          new Text(String.valueOf(totalTimeAL2)));
-      output.collect(new Text("l:numOfExceptions"), 
-          new Text(String.valueOf(numOfExceptions)));
-      output.collect(new Text("l:successfulFileOps"), 
-          new Text(String.valueOf(successfulFileOps)));
-      output.collect(new Text("l:totalTimeTPmS"), 
-              new Text(String.valueOf(totalTimeTPmS)));
-      output.collect(new Text("min:mapStartTimeTPmS"), 
-          new Text(String.valueOf(startTimeTPmS)));
-      output.collect(new Text("max:mapEndTimeTPmS"), 
-          new Text(String.valueOf(endTimeTPms)));
-    }
-    
-    /**
-     * Create and Write operation.
-     * @param name of the prefix of the putput file to be created
-     * @param reporter an instanse of (@link Reporter) to be used for
-     *   status' updates
-     */
-    private void doCreateWriteOp(String name,
-                                 Reporter reporter) {
-      FSDataOutputStream out;
-      byte[] buffer = new byte[bytesToWrite];
-      
-      for (long l = 0l; l < numberOfFiles; l++) {
-        Path filePath = new Path(new Path(baseDir, dataDirName), 
-                name + "_" + l);
-
-        boolean successfulOp = false;
-        while (! successfulOp && numOfExceptions < MAX_OPERATION_EXCEPTIONS) {
-          try {
-            // Set up timer for measuring AL (transaction #1)
-            startTimeAL = System.currentTimeMillis();
-            // Create the file
-            // Use a buffer size of 512
-            out = filesystem.create(filePath, 
-                    true, 
-                    512, 
-                    replFactor, 
-                    blkSize);
-            out.write(buffer);
-            totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
-
-            // Close the file / file output stream
-            // Set up timers for measuring AL (transaction #2)
-            startTimeAL = System.currentTimeMillis();
-            out.close();
-            
-            totalTimeAL2 += (System.currentTimeMillis() - startTimeAL);
-            successfulOp = true;
-            successfulFileOps ++;
-
-            reporter.setStatus("Finish "+ l + " files");
-          } catch (IOException e) {
-            LOG.info("Exception recorded in op: " +
-                    "Create/Write/Close");
- 
-            numOfExceptions++;
-          }
-        }
-      }
-    }
-    
-    /**
-     * Open operation
-     * @param name of the prefix of the putput file to be read
-     * @param reporter an instanse of (@link Reporter) to be used for
-     *   status' updates
-     */
-    private void doOpenReadOp(String name,
-                              Reporter reporter) {
-      FSDataInputStream input;
-      byte[] buffer = new byte[bytesToWrite];
-      
-      for (long l = 0l; l < numberOfFiles; l++) {
-        Path filePath = new Path(new Path(baseDir, dataDirName), 
-                name + "_" + l);
-
-        boolean successfulOp = false;
-        while (! successfulOp && numOfExceptions < MAX_OPERATION_EXCEPTIONS) {
-          try {
-            // Set up timer for measuring AL
-            startTimeAL = System.currentTimeMillis();
-            input = filesystem.open(filePath);
-            totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
-            
-            // If the file needs to be read (specified at command line)
-            if (readFile) {
-              startTimeAL = System.currentTimeMillis();
-              input.readFully(buffer);
-
-              totalTimeAL2 += (System.currentTimeMillis() - startTimeAL);
-            }
-            input.close();
-            successfulOp = true;
-            successfulFileOps ++;
-
-            reporter.setStatus("Finish "+ l + " files");
-          } catch (IOException e) {
-            LOG.info("Exception recorded in op: OpenRead " + e);
-            numOfExceptions++;
-          }
-        }
-      }
-    }
-    
-    /**
-     * Rename operation
-     * @param name of prefix of the file to be renamed
-     * @param reporter an instanse of (@link Reporter) to be used for
-     *   status' updates
-     */
-    private void doRenameOp(String name,
-                            Reporter reporter) {
-      for (long l = 0l; l < numberOfFiles; l++) {
-        Path filePath = new Path(new Path(baseDir, dataDirName), 
-                name + "_" + l);
-        Path filePathR = new Path(new Path(baseDir, dataDirName), 
-                name + "_r_" + l);
-
-        boolean successfulOp = false;
-        while (! successfulOp && numOfExceptions < MAX_OPERATION_EXCEPTIONS) {
-          try {
-            // Set up timer for measuring AL
-            startTimeAL = System.currentTimeMillis();
-            filesystem.rename(filePath, filePathR);
-            totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
-            
-            successfulOp = true;
-            successfulFileOps ++;
-
-            reporter.setStatus("Finish "+ l + " files");
-          } catch (IOException e) {
-            LOG.info("Exception recorded in op: Rename");
-
-            numOfExceptions++;
-          }
-        }
-      }
-    }
-    
-    /**
-     * Delete operation
-     * @param name of prefix of the file to be deleted
-     * @param reporter an instanse of (@link Reporter) to be used for
-     *   status' updates
-     */
-    private void doDeleteOp(String name,
-                            Reporter reporter) {
-      for (long l = 0l; l < numberOfFiles; l++) {
-        Path filePath = new Path(new Path(baseDir, dataDirName), 
-                name + "_" + l);
-        
-        boolean successfulOp = false;
-        while (! successfulOp && numOfExceptions < MAX_OPERATION_EXCEPTIONS) {
-          try {
-            // Set up timer for measuring AL
-            startTimeAL = System.currentTimeMillis();
-            filesystem.delete(filePath, true);
-            totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
-            
-            successfulOp = true;
-            successfulFileOps ++;
-
-            reporter.setStatus("Finish "+ l + " files");
-          } catch (IOException e) {
-            LOG.info("Exception in recorded op: Delete");
-
-            numOfExceptions++;
-          }
-        }
-      }
-    }
-  }
-  
-  /**
-   * Reducer class
-   */
-  static class NNBenchReducer extends MapReduceBase
-      implements Reducer<Text, Text, Text, Text> {
-
-    protected String hostName;
-
-    public NNBenchReducer () {
-      LOG.info("Starting NNBenchReducer !!!");
-      try {
-        hostName = java.net.InetAddress.getLocalHost().getHostName();
-      } catch(Exception e) {
-        hostName = "localhost";
-      }
-      LOG.info("Starting NNBenchReducer on " + hostName);
-    }
-
-    /**
-     * Reduce method
-     */
-    public void reduce(Text key, 
-                       Iterator<Text> values,
-                       OutputCollector<Text, Text> output, 
-                       Reporter reporter
-                       ) throws IOException {
-      String field = key.toString();
-      
-      reporter.setStatus("starting " + field + " ::host = " + hostName);
-      
-      // sum long values
-      if (field.startsWith("l:")) {
-        long lSum = 0;
-        while (values.hasNext()) {
-          lSum += Long.parseLong(values.next().toString());
-        }
-        output.collect(key, new Text(String.valueOf(lSum)));
-      }
-      
-      if (field.startsWith("min:")) {
-        long minVal = -1;
-        while (values.hasNext()) {
-          long value = Long.parseLong(values.next().toString());
-          
-          if (minVal == -1) {
-            minVal = value;
-          } else {
-            if (value != 0 && value < minVal) {
-              minVal = value;
-            }
-          }
-        }
-        output.collect(key, new Text(String.valueOf(minVal)));
-      }
-      
-      if (field.startsWith("max:")) {
-        long maxVal = -1;
-        while (values.hasNext()) {
-          long value = Long.parseLong(values.next().toString());
-          
-          if (maxVal == -1) {
-            maxVal = value;
-          } else {
-            if (value > maxVal) {
-              maxVal = value;
-            }
-          }
-        }
-        output.collect(key, new Text(String.valueOf(maxVal)));
-      }
-      
-      reporter.setStatus("finished " + field + " ::host = " + hostName);
-    }
-  }
-}

+ 0 - 359
src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBenchWithoutMR.java

@@ -1,359 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.util.Date;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.mapred.JobConf;
-
-/**
- * This program executes a specified operation that applies load to 
- * the NameNode. Possible operations include create/writing files,
- * opening/reading files, renaming files, and deleting files.
- * 
- * When run simultaneously on multiple nodes, this program functions 
- * as a stress-test and benchmark for namenode, especially when 
- * the number of bytes written to each file is small.
- * 
- * This version does not use the map reduce framework
- * 
- */
-public class NNBenchWithoutMR {
-  
-  private static final Log LOG = LogFactory.getLog(
-                                            "org.apache.hadoop.hdfs.NNBench");
-  
-  // variable initialzed from command line arguments
-  private static long startTime = 0;
-  private static int numFiles = 0;
-  private static long bytesPerBlock = 1;
-  private static long blocksPerFile = 0;
-  private static long bytesPerFile = 1;
-  private static Path baseDir = null;
-    
-  // variables initialized in main()
-  private static FileSystem fileSys = null;
-  private static Path taskDir = null;
-  private static byte[] buffer;
-  private static long maxExceptionsPerFile = 200;
-    
-  /**
-   * Returns when the current number of seconds from the epoch equals
-   * the command line argument given by <code>-startTime</code>.
-   * This allows multiple instances of this program, running on clock
-   * synchronized nodes, to start at roughly the same time.
-   */
-
-  static void barrier() {
-    long sleepTime;
-    while ((sleepTime = startTime - System.currentTimeMillis()) > 0) {
-      try {
-        Thread.sleep(sleepTime);
-      } catch (InterruptedException ex) {
-        //This left empty on purpose
-      }
-    }
-  }
-    
-  static private void handleException(String operation, Throwable e, 
-                                      int singleFileExceptions) {
-    LOG.warn("Exception while " + operation + ": " +
-             StringUtils.stringifyException(e));
-    if (singleFileExceptions >= maxExceptionsPerFile) {
-      throw new RuntimeException(singleFileExceptions + 
-        " exceptions for a single file exceeds threshold. Aborting");
-    }
-  }
-  
-  /**
-   * Create and write to a given number of files.  Repeat each remote
-   * operation until is suceeds (does not throw an exception).
-   *
-   * @return the number of exceptions caught
-   */
-  static int createWrite() {
-    int totalExceptions = 0;
-    FSDataOutputStream out = null;
-    boolean success;
-    for (int index = 0; index < numFiles; index++) {
-      int singleFileExceptions = 0;
-      do { // create file until is succeeds or max exceptions reached
-        try {
-          out = fileSys.create(
-                  new Path(taskDir, "" + index), false, 512,
-                  (short)1, bytesPerBlock);
-          success = true;
-        } catch (IOException ioe) { 
-          success=false; 
-          totalExceptions++;
-          handleException("creating file #" + index, ioe,
-                  ++singleFileExceptions);
-        }
-      } while (!success);
-      long toBeWritten = bytesPerFile;
-      while (toBeWritten > 0) {
-        int nbytes = (int) Math.min(buffer.length, toBeWritten);
-        toBeWritten -= nbytes;
-        try { // only try once
-          out.write(buffer, 0, nbytes);
-        } catch (IOException ioe) {
-          totalExceptions++;
-          handleException("writing to file #" + index, ioe,
-                  ++singleFileExceptions);
-        }
-      }
-      do { // close file until is succeeds
-        try {
-          out.close();
-          success = true;
-        } catch (IOException ioe) {
-          success=false; 
-          totalExceptions++;
-          handleException("closing file #" + index, ioe,
-                  ++singleFileExceptions);
-        }
-      } while (!success);
-    }
-    return totalExceptions;
-  }
-    
-  /**
-   * Open and read a given number of files.
-   *
-   * @return the number of exceptions caught
-   */
-  static int openRead() {
-    int totalExceptions = 0;
-    FSDataInputStream in;
-    for (int index = 0; index < numFiles; index++) {
-      int singleFileExceptions = 0;
-      try {
-        in = fileSys.open(new Path(taskDir, "" + index), 512);
-        long toBeRead = bytesPerFile;
-        while (toBeRead > 0) {
-          int nbytes = (int) Math.min(buffer.length, toBeRead);
-          toBeRead -= nbytes;
-          try { // only try once && we don't care about a number of bytes read
-            in.read(buffer, 0, nbytes);
-          } catch (IOException ioe) {
-            totalExceptions++;
-            handleException("reading from file #" + index, ioe,
-                    ++singleFileExceptions);
-          }
-        }
-        in.close();
-      } catch (IOException ioe) { 
-        totalExceptions++;
-        handleException("opening file #" + index, ioe, ++singleFileExceptions);
-      }
-    }
-    return totalExceptions;
-  }
-    
-  /**
-   * Rename a given number of files.  Repeat each remote
-   * operation until is suceeds (does not throw an exception).
-   *
-   * @return the number of exceptions caught
-   */
-  static int rename() {
-    int totalExceptions = 0;
-    boolean success;
-    for (int index = 0; index < numFiles; index++) {
-      int singleFileExceptions = 0;
-      do { // rename file until is succeeds
-        try {
-          // Possible result of this operation is at no interest to us for it
-          // can return false only if the namesystem
-          // could rename the path from the name
-          // space (e.g. no Exception has been thrown)
-          fileSys.rename(new Path(taskDir, "" + index),
-              new Path(taskDir, "A" + index));
-          success = true;
-        } catch (IOException ioe) {
-          success = false;
-          totalExceptions++;
-          handleException("creating file #" + index, ioe, ++singleFileExceptions);
-        }
-      } while (!success);
-    }
-    return totalExceptions;
-  }
-    
-  /**
-   * Delete a given number of files.  Repeat each remote
-   * operation until is suceeds (does not throw an exception).
-   *
-   * @return the number of exceptions caught
-   */
-  static int delete() {
-    int totalExceptions = 0;
-    boolean success;
-    for (int index = 0; index < numFiles; index++) {
-      int singleFileExceptions = 0;
-      do { // delete file until is succeeds
-        try {
-          // Possible result of this operation is at no interest to us for it
-          // can return false only if namesystem
-          // delete could remove the path from the name
-          // space (e.g. no Exception has been thrown)
-          fileSys.delete(new Path(taskDir, "A" + index), true);
-          success = true;
-        } catch (IOException ioe) {
-          success=false; 
-          totalExceptions++;
-          handleException("creating file #" + index, ioe, ++singleFileExceptions);
-        }
-      } while (!success);
-    }
-    return totalExceptions;
-  }
-    
-  /**
-   * This launches a given namenode operation (<code>-operation</code>),
-   * starting at a given time (<code>-startTime</code>).  The files used
-   * by the openRead, rename, and delete operations are the same files
-   * created by the createWrite operation.  Typically, the program
-   * would be run four times, once for each operation in this order:
-   * createWrite, openRead, rename, delete.
-   *
-   * <pre>
-   * Usage: nnbench 
-   *          -operation <one of createWrite, openRead, rename, or delete>
-   *          -baseDir <base output/input DFS path>
-   *          -startTime <time to start, given in seconds from the epoch>
-   *          -numFiles <number of files to create, read, rename, or delete>
-   *          -blocksPerFile <number of blocks to create per file>
-   *         [-bytesPerBlock <number of bytes to write to each block, default is 1>]
-   *         [-bytesPerChecksum <value for io.bytes.per.checksum>]
-   * </pre>
-   *
-   * @param args is an array of the program command line arguments
-   * @throws IOException indicates a problem with test startup
-   */
-  public static void main(String[] args) throws IOException {
-    String version = "NameNodeBenchmark.0.3";
-    System.out.println(version);
-    int bytesPerChecksum = -1;
-    
-    String usage =
-      "Usage: nnbench " +
-      "  -operation <one of createWrite, openRead, rename, or delete> " +
-      "  -baseDir <base output/input DFS path> " +
-      "  -startTime <time to start, given in seconds from the epoch> " +
-      "  -numFiles <number of files to create> " +
-      "  -blocksPerFile <number of blocks to create per file> " +
-      "  [-bytesPerBlock <number of bytes to write to each block, default is 1>] " +
-      "  [-bytesPerChecksum <value for io.bytes.per.checksum>]" +
-      "Note: bytesPerBlock MUST be a multiple of bytesPerChecksum";
-    
-    String operation = null;
-    for (int i = 0; i < args.length; i++) { // parse command line
-      if (args[i].equals("-baseDir")) {
-        baseDir = new Path(args[++i]);
-      } else if (args[i].equals("-numFiles")) {
-        numFiles = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-blocksPerFile")) {
-        blocksPerFile = Integer.parseInt(args[++i]);
-      } else if (args[i].equals("-bytesPerBlock")) {
-        bytesPerBlock = Long.parseLong(args[++i]);
-      } else if (args[i].equals("-bytesPerChecksum")) {
-        bytesPerChecksum = Integer.parseInt(args[++i]);        
-      } else if (args[i].equals("-startTime")) {
-        startTime = Long.parseLong(args[++i]) * 1000;
-      } else if (args[i].equals("-operation")) {
-        operation = args[++i];
-      } else {
-        System.out.println(usage);
-        System.exit(-1);
-      }
-    }
-    bytesPerFile = bytesPerBlock * blocksPerFile;
-    
-    JobConf jobConf = new JobConf(new Configuration(), NNBench.class);
-
-    if ( bytesPerChecksum < 0 ) { // if it is not set in cmdline
-      bytesPerChecksum = jobConf.getInt("io.bytes.per.checksum", 512);
-    }
-    jobConf.set("io.bytes.per.checksum", Integer.toString(bytesPerChecksum));
-    
-    System.out.println("Inputs: ");
-    System.out.println("   operation: " + operation);
-    System.out.println("   baseDir: " + baseDir);
-    System.out.println("   startTime: " + startTime);
-    System.out.println("   numFiles: " + numFiles);
-    System.out.println("   blocksPerFile: " + blocksPerFile);
-    System.out.println("   bytesPerBlock: " + bytesPerBlock);
-    System.out.println("   bytesPerChecksum: " + bytesPerChecksum);
-    
-    if (operation == null ||  // verify args
-        baseDir == null ||
-        numFiles < 1 ||
-        blocksPerFile < 1 ||
-        bytesPerBlock < 0 ||
-        bytesPerBlock % bytesPerChecksum != 0)
-      {
-        System.err.println(usage);
-        System.exit(-1);
-      }
-    
-    fileSys = FileSystem.get(jobConf);
-    String uniqueId = java.net.InetAddress.getLocalHost().getHostName();
-    taskDir = new Path(baseDir, uniqueId);
-    // initialize buffer used for writing/reading file
-    buffer = new byte[(int) Math.min(bytesPerFile, 32768L)];
-    
-    Date execTime;
-    Date endTime;
-    long duration;
-    int exceptions = 0;
-    barrier(); // wait for coordinated start time
-    execTime = new Date();
-    System.out.println("Job started: " + startTime);
-    if (operation.equals("createWrite")) {
-      if (!fileSys.mkdirs(taskDir)) {
-        throw new IOException("Mkdirs failed to create " + taskDir.toString());
-      }
-      exceptions = createWrite();
-    } else if (operation.equals("openRead")) {
-      exceptions = openRead();
-    } else if (operation.equals("rename")) {
-      exceptions = rename();
-    } else if (operation.equals("delete")) {
-      exceptions = delete();
-    } else {
-      System.err.println(usage);
-      System.exit(-1);
-    }
-    endTime = new Date();
-    System.out.println("Job ended: " + endTime);
-    duration = (endTime.getTime() - execTime.getTime()) /1000;
-    System.out.println("The " + operation + " job took " + duration + " seconds.");
-    System.out.println("The job recorded " + exceptions + " exceptions.");
-  }
-}

+ 0 - 603
src/test/hdfs-with-mr/org/apache/hadoop/io/FileBench.java

@@ -1,603 +0,0 @@
-/**
- * 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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-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.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class FileBench extends Configured implements Tool {
-
-  static int printUsage() {
-    ToolRunner.printGenericCommandUsage(System.out);
-    System.out.println(
-"Usage: Task list:           -[no]r -[no]w\n" +
-"       Format:              -[no]seq -[no]txt\n" +
-"       CompressionCodec:    -[no]zip -[no]pln\n" +
-"       CompressionType:     -[no]blk -[no]rec\n" +
-"       Required:            -dir <working dir>\n" +
-"All valid combinations are implicitly enabled, unless an option is enabled\n" +
-"explicitly. For example, specifying \"-zip\", excludes -pln,\n" +
-"unless they are also explicitly included, as in \"-pln -zip\"\n" +
-"Note that CompressionType params only apply to SequenceFiles\n\n" +
-"Useful options to set:\n" +
-"-D fs.default.name=\"file:///\" \\\n" +
-"-D fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem \\\n" +
-"-D filebench.file.bytes=$((10*1024*1024*1024)) \\\n" +
-"-D filebench.key.words=5 \\\n" +
-"-D filebench.val.words=20\n");
-    return -1;
-  }
-
-  static String[] keys;
-  static String[] values;
-  static StringBuilder sentence = new StringBuilder();
-
-  private static String generateSentence(Random r, int noWords) {
-    sentence.setLength(0);
-    for (int i=0; i < noWords; ++i) {
-      sentence.append(words[r.nextInt(words.length)]);
-      sentence.append(" ");
-    }
-    return sentence.toString();
-  }
-
-  // fill keys, values with ~1.5 blocks for block-compressed seq fill
-  private static void fillBlocks(JobConf conf) {
-    Random r = new Random();
-    long seed = conf.getLong("filebench.seed", -1);
-    if (seed > 0) {
-      r.setSeed(seed);
-    }
-
-    int keylen = conf.getInt("filebench.key.words", 5);
-    int vallen = conf.getInt("filebench.val.words", 20);
-    int acc = (3 * conf.getInt("io.seqfile.compress.blocksize", 1000000)) >> 1;
-    ArrayList<String> k = new ArrayList<String>();
-    ArrayList<String> v = new ArrayList<String>();
-    for (int i = 0; acc > 0; ++i) {
-      String s = generateSentence(r, keylen);
-      acc -= s.length();
-      k.add(s);
-      s = generateSentence(r, vallen);
-      acc -= s.length();
-      v.add(s);
-    }
-    keys = k.toArray(new String[0]);
-    values = v.toArray(new String[0]);
-  }
-
-  @SuppressWarnings("unchecked") // OutputFormat instantiation
-  static long writeBench(JobConf conf) throws IOException {
-    long filelen = conf.getLong("filebench.file.bytes", 5 * 1024 * 1024 * 1024);
-    Text key = new Text();
-    Text val = new Text();
-
-    final String fn = conf.get("test.filebench.name", "");
-    final Path outd = FileOutputFormat.getOutputPath(conf);
-    conf.set("mapred.work.output.dir", outd.toString());
-    OutputFormat outf = conf.getOutputFormat();
-    RecordWriter<Text,Text> rw =
-      outf.getRecordWriter(outd.getFileSystem(conf), conf, fn,
-                           Reporter.NULL);
-    try {
-      long acc = 0L;
-      Date start = new Date();
-      for (int i = 0; acc < filelen; ++i) {
-        i %= keys.length;
-        key.set(keys[i]);
-        val.set(values[i]);
-        rw.write(key, val);
-        acc += keys[i].length();
-        acc += values[i].length();
-      }
-      Date end = new Date();
-      return end.getTime() - start.getTime();
-    } finally {
-      rw.close(Reporter.NULL);
-    }
-  }
-
-  @SuppressWarnings("unchecked") // InputFormat instantiation
-  static long readBench(JobConf conf) throws IOException {
-    InputFormat inf = conf.getInputFormat();
-    final String fn = conf.get("test.filebench.name", "");
-    Path pin = new Path(FileInputFormat.getInputPaths(conf)[0], fn);
-    FileStatus in = pin.getFileSystem(conf).getFileStatus(pin);
-    RecordReader rr = inf.getRecordReader(new FileSplit(pin, 0, in.getLen(), 
-                                          (String[])null), conf, Reporter.NULL);
-    try {
-      Object key = rr.createKey();
-      Object val = rr.createValue();
-      Date start = new Date();
-      while (rr.next(key, val));
-      Date end = new Date();
-      return end.getTime() - start.getTime();
-    } finally {
-      rr.close();
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new FileBench(), args);
-    System.exit(res);
-  }
-
-  /**
-   * Process params from command line and run set of benchmarks specified.
-   */
-  public int run(String[] argv) throws IOException {
-    JobConf job = new JobConf(getConf());
-    EnumSet<CCodec> cc = null;
-    EnumSet<CType> ct = null;
-    EnumSet<Format> f = null;
-    EnumSet<RW> rw = null;
-    Path root = null;
-    FileSystem fs = FileSystem.get(job);
-    for(int i = 0; i < argv.length; ++i) {
-      try {
-        if ("-dir".equals(argv[i])) {
-          root = new Path(argv[++i]).makeQualified(fs);
-          System.out.println("DIR: " + root.toString());
-        } else if ("-seed".equals(argv[i])) {
-          job.setLong("filebench.seed", Long.valueOf(argv[++i]));
-        } else if (argv[i].startsWith("-no")) {
-          String arg = argv[i].substring(3);
-          cc = rem(CCodec.class, cc, arg);
-          ct = rem(CType.class, ct, arg);
-          f =  rem(Format.class, f, arg);
-          rw = rem(RW.class, rw, arg);
-        } else {
-          String arg = argv[i].substring(1);
-          cc = add(CCodec.class, cc, arg);
-          ct = add(CType.class, ct, arg);
-          f =  add(Format.class, f, arg);
-          rw = add(RW.class, rw, arg);
-        }
-      } catch (Exception e) {
-        throw (IOException)new IOException().initCause(e);
-      }
-    }
-    if (null == root) {
-      System.out.println("Missing -dir param");
-      printUsage();
-      return -1;
-    }
-
-    fillBlocks(job);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    FileInputFormat.setInputPaths(job, root);
-    FileOutputFormat.setOutputPath(job, root);
-
-    if (null == cc) cc = EnumSet.allOf(CCodec.class);
-    if (null == ct) ct = EnumSet.allOf(CType.class);
-    if (null == f)  f  = EnumSet.allOf(Format.class);
-    if (null == rw) rw = EnumSet.allOf(RW.class);
-    for (RW rwop : rw) {
-      for (Format fmt : f) {
-        fmt.configure(job);
-        for (CCodec cod : cc) {
-          cod.configure(job);
-          if (!(fmt == Format.txt || cod == CCodec.pln)) {
-            for (CType typ : ct) {
-              String fn =
-                fmt.name().toUpperCase() + "_" +
-                cod.name().toUpperCase() + "_" +
-                typ.name().toUpperCase();
-              typ.configure(job);
-              System.out.print(rwop.name().toUpperCase() + " " + fn + ": ");
-              System.out.println(rwop.exec(fn, job) / 1000 +
-                  " seconds");
-            }
-          } else {
-            String fn =
-              fmt.name().toUpperCase() + "_" +
-              cod.name().toUpperCase();
-            Path p = new Path(root, fn);
-            if (rwop == RW.r && !fs.exists(p)) {
-              fn += cod.getExt();
-            }
-            System.out.print(rwop.name().toUpperCase() + " " + fn + ": ");
-            System.out.println(rwop.exec(fn, job) / 1000 +
-                " seconds");
-          }
-        }
-      }
-    }
-    return 0;
-  }
-
-  // overwrought argument processing and wordlist follow
-  enum CCodec {
-    zip(GzipCodec.class, ".gz"), pln(null, "");
-
-    Class<? extends CompressionCodec> inf;
-    String ext;
-    CCodec(Class<? extends CompressionCodec> inf, String ext) {
-      this.inf = inf;
-      this.ext = ext;
-    }
-    public void configure(JobConf job) {
-      if (inf != null) {
-        job.setBoolean("mapred.output.compress", true);
-        job.setClass("mapred.output.compression.codec", inf,
-            CompressionCodec.class);
-      } else {
-        job.setBoolean("mapred.output.compress", false);
-      }
-    }
-    public String getExt() { return ext; }
-  }
-  enum CType {
-    blk("BLOCK"),
-    rec("RECORD");
-
-    String typ;
-    CType(String typ) { this.typ = typ; }
-    public void configure(JobConf job) {
-      job.set("mapred.map.output.compression.type", typ);
-      job.set("mapred.output.compression.type", typ);
-    }
-  }
-  enum Format {
-    seq(SequenceFileInputFormat.class, SequenceFileOutputFormat.class),
-    txt(TextInputFormat.class, TextOutputFormat.class);
-
-    Class<? extends InputFormat> inf;
-    Class<? extends OutputFormat> of;
-    Format(Class<? extends InputFormat> inf, Class<? extends OutputFormat> of) {
-      this.inf = inf;
-      this.of = of;
-    }
-    public void configure(JobConf job) {
-      if (null != inf) job.setInputFormat(inf);
-      if (null != of) job.setOutputFormat(of);
-    }
-  }
-  enum RW {
-    w() {
-      public long exec(String fn, JobConf job) throws IOException {
-        job.set("test.filebench.name", fn);
-        return writeBench(job);
-      }
-    },
-
-    r() {
-      public long exec(String fn, JobConf job) throws IOException {
-        job.set("test.filebench.name", fn);
-        return readBench(job);
-      }
-    };
-
-    public abstract long exec(String fn, JobConf job) throws IOException;
-  }
-  static Map<Class<? extends Enum>, Map<String,? extends Enum>> fullmap
-    = new HashMap<Class<? extends Enum>, Map<String,? extends Enum>>();
-  static {
-    // can't effectively use Enum::valueOf
-    Map<String,CCodec> m1 = new HashMap<String,CCodec>();
-    for (CCodec v : CCodec.values()) m1.put(v.name(), v);
-    fullmap.put(CCodec.class, m1);
-    Map<String,CType> m2 = new HashMap<String,CType>();
-    for (CType v : CType.values()) m2.put(v.name(), v);
-    fullmap.put(CType.class, m2);
-    Map<String,Format> m3 = new HashMap<String,Format>();
-    for (Format v : Format.values()) m3.put(v.name(), v);
-    fullmap.put(Format.class, m3);
-    Map<String,RW> m4 = new HashMap<String,RW>();
-    for (RW v : RW.values()) m4.put(v.name(), v);
-    fullmap.put(RW.class, m4);
-  }
-
-  public static <T extends Enum<T>> EnumSet<T> rem(Class<T> c,
-      EnumSet<T> set, String s) {
-    if (null != fullmap.get(c) && fullmap.get(c).get(s) != null) {
-      if (null == set) {
-        set = EnumSet.allOf(c);
-      }
-      set.remove(fullmap.get(c).get(s));
-    }
-    return set;
-  }
-
-  @SuppressWarnings("unchecked")
-  public static <T extends Enum<T>> EnumSet<T> add(Class<T> c,
-      EnumSet<T> set, String s) {
-    if (null != fullmap.get(c) && fullmap.get(c).get(s) != null) {
-      if (null == set) {
-        set = EnumSet.noneOf(c);
-      }
-      set.add((T)fullmap.get(c).get(s));
-    }
-    return set;
-  }
-
-  /**
-   * A random list of 1000 words from /usr/share/dict/words
-   */
-  private static final String[] words = {
-    "diurnalness", "Homoiousian", "spiranthic", "tetragynian",
-    "silverhead", "ungreat", "lithograph", "exploiter",
-    "physiologian", "by", "hellbender", "Filipendula",
-    "undeterring", "antiscolic", "pentagamist", "hypoid",
-    "cacuminal", "sertularian", "schoolmasterism", "nonuple",
-    "gallybeggar", "phytonic", "swearingly", "nebular",
-    "Confervales", "thermochemically", "characinoid", "cocksuredom",
-    "fallacious", "feasibleness", "debromination", "playfellowship",
-    "tramplike", "testa", "participatingly", "unaccessible",
-    "bromate", "experientialist", "roughcast", "docimastical",
-    "choralcelo", "blightbird", "peptonate", "sombreroed",
-    "unschematized", "antiabolitionist", "besagne", "mastication",
-    "bromic", "sviatonosite", "cattimandoo", "metaphrastical",
-    "endotheliomyoma", "hysterolysis", "unfulminated", "Hester",
-    "oblongly", "blurredness", "authorling", "chasmy",
-    "Scorpaenidae", "toxihaemia", "Dictograph", "Quakerishly",
-    "deaf", "timbermonger", "strammel", "Thraupidae",
-    "seditious", "plerome", "Arneb", "eristically",
-    "serpentinic", "glaumrie", "socioromantic", "apocalypst",
-    "tartrous", "Bassaris", "angiolymphoma", "horsefly",
-    "kenno", "astronomize", "euphemious", "arsenide",
-    "untongued", "parabolicness", "uvanite", "helpless",
-    "gemmeous", "stormy", "templar", "erythrodextrin",
-    "comism", "interfraternal", "preparative", "parastas",
-    "frontoorbital", "Ophiosaurus", "diopside", "serosanguineous",
-    "ununiformly", "karyological", "collegian", "allotropic",
-    "depravity", "amylogenesis", "reformatory", "epidymides",
-    "pleurotropous", "trillium", "dastardliness", "coadvice",
-    "embryotic", "benthonic", "pomiferous", "figureheadship",
-    "Megaluridae", "Harpa", "frenal", "commotion",
-    "abthainry", "cobeliever", "manilla", "spiciferous",
-    "nativeness", "obispo", "monilioid", "biopsic",
-    "valvula", "enterostomy", "planosubulate", "pterostigma",
-    "lifter", "triradiated", "venialness", "tum",
-    "archistome", "tautness", "unswanlike", "antivenin",
-    "Lentibulariaceae", "Triphora", "angiopathy", "anta",
-    "Dawsonia", "becomma", "Yannigan", "winterproof",
-    "antalgol", "harr", "underogating", "ineunt",
-    "cornberry", "flippantness", "scyphostoma", "approbation",
-    "Ghent", "Macraucheniidae", "scabbiness", "unanatomized",
-    "photoelasticity", "eurythermal", "enation", "prepavement",
-    "flushgate", "subsequentially", "Edo", "antihero",
-    "Isokontae", "unforkedness", "porriginous", "daytime",
-    "nonexecutive", "trisilicic", "morphiomania", "paranephros",
-    "botchedly", "impugnation", "Dodecatheon", "obolus",
-    "unburnt", "provedore", "Aktistetae", "superindifference",
-    "Alethea", "Joachimite", "cyanophilous", "chorograph",
-    "brooky", "figured", "periclitation", "quintette",
-    "hondo", "ornithodelphous", "unefficient", "pondside",
-    "bogydom", "laurinoxylon", "Shiah", "unharmed",
-    "cartful", "noncrystallized", "abusiveness", "cromlech",
-    "japanned", "rizzomed", "underskin", "adscendent",
-    "allectory", "gelatinousness", "volcano", "uncompromisingly",
-    "cubit", "idiotize", "unfurbelowed", "undinted",
-    "magnetooptics", "Savitar", "diwata", "ramosopalmate",
-    "Pishquow", "tomorn", "apopenptic", "Haversian",
-    "Hysterocarpus", "ten", "outhue", "Bertat",
-    "mechanist", "asparaginic", "velaric", "tonsure",
-    "bubble", "Pyrales", "regardful", "glyphography",
-    "calabazilla", "shellworker", "stradametrical", "havoc",
-    "theologicopolitical", "sawdust", "diatomaceous", "jajman",
-    "temporomastoid", "Serrifera", "Ochnaceae", "aspersor",
-    "trailmaking", "Bishareen", "digitule", "octogynous",
-    "epididymitis", "smokefarthings", "bacillite", "overcrown",
-    "mangonism", "sirrah", "undecorated", "psychofugal",
-    "bismuthiferous", "rechar", "Lemuridae", "frameable",
-    "thiodiazole", "Scanic", "sportswomanship", "interruptedness",
-    "admissory", "osteopaedion", "tingly", "tomorrowness",
-    "ethnocracy", "trabecular", "vitally", "fossilism",
-    "adz", "metopon", "prefatorial", "expiscate",
-    "diathermacy", "chronist", "nigh", "generalizable",
-    "hysterogen", "aurothiosulphuric", "whitlowwort", "downthrust",
-    "Protestantize", "monander", "Itea", "chronographic",
-    "silicize", "Dunlop", "eer", "componental",
-    "spot", "pamphlet", "antineuritic", "paradisean",
-    "interruptor", "debellator", "overcultured", "Florissant",
-    "hyocholic", "pneumatotherapy", "tailoress", "rave",
-    "unpeople", "Sebastian", "thermanesthesia", "Coniferae",
-    "swacking", "posterishness", "ethmopalatal", "whittle",
-    "analgize", "scabbardless", "naught", "symbiogenetically",
-    "trip", "parodist", "columniform", "trunnel",
-    "yawler", "goodwill", "pseudohalogen", "swangy",
-    "cervisial", "mediateness", "genii", "imprescribable",
-    "pony", "consumptional", "carposporangial", "poleax",
-    "bestill", "subfebrile", "sapphiric", "arrowworm",
-    "qualminess", "ultraobscure", "thorite", "Fouquieria",
-    "Bermudian", "prescriber", "elemicin", "warlike",
-    "semiangle", "rotular", "misthread", "returnability",
-    "seraphism", "precostal", "quarried", "Babylonism",
-    "sangaree", "seelful", "placatory", "pachydermous",
-    "bozal", "galbulus", "spermaphyte", "cumbrousness",
-    "pope", "signifier", "Endomycetaceae", "shallowish",
-    "sequacity", "periarthritis", "bathysphere", "pentosuria",
-    "Dadaism", "spookdom", "Consolamentum", "afterpressure",
-    "mutter", "louse", "ovoviviparous", "corbel",
-    "metastoma", "biventer", "Hydrangea", "hogmace",
-    "seizing", "nonsuppressed", "oratorize", "uncarefully",
-    "benzothiofuran", "penult", "balanocele", "macropterous",
-    "dishpan", "marten", "absvolt", "jirble",
-    "parmelioid", "airfreighter", "acocotl", "archesporial",
-    "hypoplastral", "preoral", "quailberry", "cinque",
-    "terrestrially", "stroking", "limpet", "moodishness",
-    "canicule", "archididascalian", "pompiloid", "overstaid",
-    "introducer", "Italical", "Christianopaganism", "prescriptible",
-    "subofficer", "danseuse", "cloy", "saguran",
-    "frictionlessly", "deindividualization", "Bulanda", "ventricous",
-    "subfoliar", "basto", "scapuloradial", "suspend",
-    "stiffish", "Sphenodontidae", "eternal", "verbid",
-    "mammonish", "upcushion", "barkometer", "concretion",
-    "preagitate", "incomprehensible", "tristich", "visceral",
-    "hemimelus", "patroller", "stentorophonic", "pinulus",
-    "kerykeion", "brutism", "monstership", "merciful",
-    "overinstruct", "defensibly", "bettermost", "splenauxe",
-    "Mormyrus", "unreprimanded", "taver", "ell",
-    "proacquittal", "infestation", "overwoven", "Lincolnlike",
-    "chacona", "Tamil", "classificational", "lebensraum",
-    "reeveland", "intuition", "Whilkut", "focaloid",
-    "Eleusinian", "micromembrane", "byroad", "nonrepetition",
-    "bacterioblast", "brag", "ribaldrous", "phytoma",
-    "counteralliance", "pelvimetry", "pelf", "relaster",
-    "thermoresistant", "aneurism", "molossic", "euphonym",
-    "upswell", "ladhood", "phallaceous", "inertly",
-    "gunshop", "stereotypography", "laryngic", "refasten",
-    "twinling", "oflete", "hepatorrhaphy", "electrotechnics",
-    "cockal", "guitarist", "topsail", "Cimmerianism",
-    "larklike", "Llandovery", "pyrocatechol", "immatchable",
-    "chooser", "metrocratic", "craglike", "quadrennial",
-    "nonpoisonous", "undercolored", "knob", "ultratense",
-    "balladmonger", "slait", "sialadenitis", "bucketer",
-    "magnificently", "unstipulated", "unscourged", "unsupercilious",
-    "packsack", "pansophism", "soorkee", "percent",
-    "subirrigate", "champer", "metapolitics", "spherulitic",
-    "involatile", "metaphonical", "stachyuraceous", "speckedness",
-    "bespin", "proboscidiform", "gul", "squit",
-    "yeelaman", "peristeropode", "opacousness", "shibuichi",
-    "retinize", "yote", "misexposition", "devilwise",
-    "pumpkinification", "vinny", "bonze", "glossing",
-    "decardinalize", "transcortical", "serphoid", "deepmost",
-    "guanajuatite", "wemless", "arval", "lammy",
-    "Effie", "Saponaria", "tetrahedral", "prolificy",
-    "excerpt", "dunkadoo", "Spencerism", "insatiately",
-    "Gilaki", "oratorship", "arduousness", "unbashfulness",
-    "Pithecolobium", "unisexuality", "veterinarian", "detractive",
-    "liquidity", "acidophile", "proauction", "sural",
-    "totaquina", "Vichyite", "uninhabitedness", "allegedly",
-    "Gothish", "manny", "Inger", "flutist",
-    "ticktick", "Ludgatian", "homotransplant", "orthopedical",
-    "diminutively", "monogoneutic", "Kenipsim", "sarcologist",
-    "drome", "stronghearted", "Fameuse", "Swaziland",
-    "alen", "chilblain", "beatable", "agglomeratic",
-    "constitutor", "tendomucoid", "porencephalous", "arteriasis",
-    "boser", "tantivy", "rede", "lineamental",
-    "uncontradictableness", "homeotypical", "masa", "folious",
-    "dosseret", "neurodegenerative", "subtransverse", "Chiasmodontidae",
-    "palaeotheriodont", "unstressedly", "chalcites", "piquantness",
-    "lampyrine", "Aplacentalia", "projecting", "elastivity",
-    "isopelletierin", "bladderwort", "strander", "almud",
-    "iniquitously", "theologal", "bugre", "chargeably",
-    "imperceptivity", "meriquinoidal", "mesophyte", "divinator",
-    "perfunctory", "counterappellant", "synovial", "charioteer",
-    "crystallographical", "comprovincial", "infrastapedial", "pleasurehood",
-    "inventurous", "ultrasystematic", "subangulated", "supraoesophageal",
-    "Vaishnavism", "transude", "chrysochrous", "ungrave",
-    "reconciliable", "uninterpleaded", "erlking", "wherefrom",
-    "aprosopia", "antiadiaphorist", "metoxazine", "incalculable",
-    "umbellic", "predebit", "foursquare", "unimmortal",
-    "nonmanufacture", "slangy", "predisputant", "familist",
-    "preaffiliate", "friarhood", "corelysis", "zoonitic",
-    "halloo", "paunchy", "neuromimesis", "aconitine",
-    "hackneyed", "unfeeble", "cubby", "autoschediastical",
-    "naprapath", "lyrebird", "inexistency", "leucophoenicite",
-    "ferrogoslarite", "reperuse", "uncombable", "tambo",
-    "propodiale", "diplomatize", "Russifier", "clanned",
-    "corona", "michigan", "nonutilitarian", "transcorporeal",
-    "bought", "Cercosporella", "stapedius", "glandularly",
-    "pictorially", "weism", "disilane", "rainproof",
-    "Caphtor", "scrubbed", "oinomancy", "pseudoxanthine",
-    "nonlustrous", "redesertion", "Oryzorictinae", "gala",
-    "Mycogone", "reappreciate", "cyanoguanidine", "seeingness",
-    "breadwinner", "noreast", "furacious", "epauliere",
-    "omniscribent", "Passiflorales", "uninductive", "inductivity",
-    "Orbitolina", "Semecarpus", "migrainoid", "steprelationship",
-    "phlogisticate", "mesymnion", "sloped", "edificator",
-    "beneficent", "culm", "paleornithology", "unurban",
-    "throbless", "amplexifoliate", "sesquiquintile", "sapience",
-    "astucious", "dithery", "boor", "ambitus",
-    "scotching", "uloid", "uncompromisingness", "hoove",
-    "waird", "marshiness", "Jerusalem", "mericarp",
-    "unevoked", "benzoperoxide", "outguess", "pyxie",
-    "hymnic", "euphemize", "mendacity", "erythremia",
-    "rosaniline", "unchatteled", "lienteria", "Bushongo",
-    "dialoguer", "unrepealably", "rivethead", "antideflation",
-    "vinegarish", "manganosiderite", "doubtingness", "ovopyriform",
-    "Cephalodiscus", "Muscicapa", "Animalivora", "angina",
-    "planispheric", "ipomoein", "cuproiodargyrite", "sandbox",
-    "scrat", "Munnopsidae", "shola", "pentafid",
-    "overstudiousness", "times", "nonprofession", "appetible",
-    "valvulotomy", "goladar", "uniarticular", "oxyterpene",
-    "unlapsing", "omega", "trophonema", "seminonflammable",
-    "circumzenithal", "starer", "depthwise", "liberatress",
-    "unleavened", "unrevolting", "groundneedle", "topline",
-    "wandoo", "umangite", "ordinant", "unachievable",
-    "oversand", "snare", "avengeful", "unexplicit",
-    "mustafina", "sonable", "rehabilitative", "eulogization",
-    "papery", "technopsychology", "impressor", "cresylite",
-    "entame", "transudatory", "scotale", "pachydermatoid",
-    "imaginary", "yeat", "slipped", "stewardship",
-    "adatom", "cockstone", "skyshine", "heavenful",
-    "comparability", "exprobratory", "dermorhynchous", "parquet",
-    "cretaceous", "vesperal", "raphis", "undangered",
-    "Glecoma", "engrain", "counteractively", "Zuludom",
-    "orchiocatabasis", "Auriculariales", "warriorwise", "extraorganismal",
-    "overbuilt", "alveolite", "tetchy", "terrificness",
-    "widdle", "unpremonished", "rebilling", "sequestrum",
-    "equiconvex", "heliocentricism", "catabaptist", "okonite",
-    "propheticism", "helminthagogic", "calycular", "giantly",
-    "wingable", "golem", "unprovided", "commandingness",
-    "greave", "haply", "doina", "depressingly",
-    "subdentate", "impairment", "decidable", "neurotrophic",
-    "unpredict", "bicorporeal", "pendulant", "flatman",
-    "intrabred", "toplike", "Prosobranchiata", "farrantly",
-    "toxoplasmosis", "gorilloid", "dipsomaniacal", "aquiline",
-    "atlantite", "ascitic", "perculsive", "prospectiveness",
-    "saponaceous", "centrifugalization", "dinical", "infravaginal",
-    "beadroll", "affaite", "Helvidian", "tickleproof",
-    "abstractionism", "enhedge", "outwealth", "overcontribute",
-    "coldfinch", "gymnastic", "Pincian", "Munychian",
-    "codisjunct", "quad", "coracomandibular", "phoenicochroite",
-    "amender", "selectivity", "putative", "semantician",
-    "lophotrichic", "Spatangoidea", "saccharogenic", "inferent",
-    "Triconodonta", "arrendation", "sheepskin", "taurocolla",
-    "bunghole", "Machiavel", "triakistetrahedral", "dehairer",
-    "prezygapophysial", "cylindric", "pneumonalgia", "sleigher",
-    "emir", "Socraticism", "licitness", "massedly",
-    "instructiveness", "sturdied", "redecrease", "starosta",
-    "evictor", "orgiastic", "squdge", "meloplasty",
-    "Tsonecan", "repealableness", "swoony", "myesthesia",
-    "molecule", "autobiographist", "reciprocation", "refective",
-    "unobservantness", "tricae", "ungouged", "floatability",
-    "Mesua", "fetlocked", "chordacentrum", "sedentariness",
-    "various", "laubanite", "nectopod", "zenick",
-    "sequentially", "analgic", "biodynamics", "posttraumatic",
-    "nummi", "pyroacetic", "bot", "redescend",
-    "dispermy", "undiffusive", "circular", "trillion",
-    "Uraniidae", "ploration", "discipular", "potentness",
-    "sud", "Hu", "Eryon", "plugger",
-    "subdrainage", "jharal", "abscission", "supermarket",
-    "countergabion", "glacierist", "lithotresis", "minniebush",
-    "zanyism", "eucalypteol", "sterilely", "unrealize",
-    "unpatched", "hypochondriacism", "critically", "cheesecutter",
-  };
-}

+ 0 - 98
src/test/hdfs-with-mr/org/apache/hadoop/io/TestSequenceFileMergeProgress.java

@@ -1,98 +0,0 @@
-/**
- * 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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile.Sorter.RawKeyValueIterator;
-import org.apache.hadoop.io.SequenceFile.Sorter.SegmentDescriptor;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.mapred.*;
-
-import junit.framework.TestCase;
-import org.apache.commons.logging.*;
-
-public class TestSequenceFileMergeProgress extends TestCase {
-  private static final Log LOG = FileInputFormat.LOG;
-  private static final int RECORDS = 10000;
-  
-  public void testMergeProgressWithNoCompression() throws IOException {
-    runTest(SequenceFile.CompressionType.NONE);
-  }
-
-  public void testMergeProgressWithRecordCompression() throws IOException {
-    runTest(SequenceFile.CompressionType.RECORD);
-  }
-
-  public void testMergeProgressWithBlockCompression() throws IOException {
-    runTest(SequenceFile.CompressionType.BLOCK);
-  }
-
-  public void runTest(CompressionType compressionType) throws IOException {
-    JobConf job = new JobConf();
-    FileSystem fs = FileSystem.getLocal(job);
-    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
-    Path file = new Path(dir, "test.seq");
-    Path tempDir = new Path(dir, "tmp");
-
-    fs.delete(dir, true);
-    FileInputFormat.setInputPaths(job, dir);
-    fs.mkdirs(tempDir);
-
-    LongWritable tkey = new LongWritable();
-    Text tval = new Text();
-
-    SequenceFile.Writer writer =
-      SequenceFile.createWriter(fs, job, file, LongWritable.class, Text.class,
-        compressionType, new DefaultCodec());
-    try {
-      for (int i = 0; i < RECORDS; ++i) {
-        tkey.set(1234);
-        tval.set("valuevaluevaluevaluevaluevaluevaluevaluevaluevaluevalue");
-        writer.append(tkey, tval);
-      }
-    } finally {
-      writer.close();
-    }
-    
-    long fileLength = fs.getFileStatus(file).getLen();
-    LOG.info("With compression = " + compressionType + ": "
-        + "compressed length = " + fileLength);
-    
-    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs, 
-        job.getOutputKeyComparator(), job.getMapOutputKeyClass(),
-        job.getMapOutputValueClass(), job);
-    Path[] paths = new Path[] {file};
-    RawKeyValueIterator rIter = sorter.merge(paths, tempDir, false);
-    int count = 0;
-    while (rIter.next()) {
-      count++;
-    }
-    assertEquals(RECORDS, count);
-    assertEquals(1.0f, rIter.getProgress().get());
-  }
-
-}

+ 0 - 197
src/test/hdfs-with-mr/org/apache/hadoop/ipc/TestSocketFactory.java

@@ -1,197 +0,0 @@
-/**
- * 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.ipc;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobStatus;
-import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.hadoop.net.StandardSocketFactory;
-
-/**
- * This class checks that RPCs can use specialized socket factories.
- */
-public class TestSocketFactory extends TestCase {
-
-  /**
-   * Check that we can reach a NameNode or a JobTracker using a specific
-   * socket factory
-   */
-  public void testSocketFactory() throws IOException {
-    // Create a standard mini-cluster
-    Configuration sconf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster(sconf, 1, true, null);
-    final int nameNodePort = cluster.getNameNodePort();
-
-    // Get a reference to its DFS directly
-    FileSystem fs = cluster.getFileSystem();
-    assertTrue(fs instanceof DistributedFileSystem);
-    DistributedFileSystem directDfs = (DistributedFileSystem) fs;
-
-    // Get another reference via network using a specific socket factory
-    Configuration cconf = new Configuration();
-    FileSystem.setDefaultUri(cconf, String.format("hdfs://localhost:%s/",
-        nameNodePort + 10));
-    cconf.set("hadoop.rpc.socket.factory.class.default",
-        "org.apache.hadoop.ipc.DummySocketFactory");
-    cconf.set("hadoop.rpc.socket.factory.class.ClientProtocol",
-        "org.apache.hadoop.ipc.DummySocketFactory");
-    cconf.set("hadoop.rpc.socket.factory.class.JobSubmissionProtocol",
-        "org.apache.hadoop.ipc.DummySocketFactory");
-
-    fs = FileSystem.get(cconf);
-    assertTrue(fs instanceof DistributedFileSystem);
-    DistributedFileSystem dfs = (DistributedFileSystem) fs;
-
-    JobClient client = null;
-    MiniMRCluster mr = null;
-    try {
-      // This will test RPC to the NameNode only.
-      // could we test Client-DataNode connections?
-      Path filePath = new Path("/dir");
-
-      assertFalse(directDfs.exists(filePath));
-      assertFalse(dfs.exists(filePath));
-
-      directDfs.mkdirs(filePath);
-      assertTrue(directDfs.exists(filePath));
-      assertTrue(dfs.exists(filePath));
-
-      // This will test TPC to a JobTracker
-      fs = FileSystem.get(sconf);
-      mr = new MiniMRCluster(1, fs.getUri().toString(), 1);
-      final int jobTrackerPort = mr.getJobTrackerPort();
-
-      JobConf jconf = new JobConf(cconf);
-      jconf.set("mapred.job.tracker", String.format("localhost:%d",
-          jobTrackerPort + 10));
-      client = new JobClient(jconf);
-
-      JobStatus[] jobs = client.jobsToComplete();
-      assertTrue(jobs.length == 0);
-
-    } finally {
-      try {
-        if (client != null)
-          client.close();
-      } catch (Exception ignored) {
-        // nothing we can do
-        ignored.printStackTrace();
-      }
-      try {
-        if (dfs != null)
-          dfs.close();
-
-      } catch (Exception ignored) {
-        // nothing we can do
-        ignored.printStackTrace();
-      }
-      try {
-        if (directDfs != null)
-          directDfs.close();
-
-      } catch (Exception ignored) {
-        // nothing we can do
-        ignored.printStackTrace();
-      }
-      try {
-        if (cluster != null)
-          cluster.shutdown();
-
-      } catch (Exception ignored) {
-        // nothing we can do
-        ignored.printStackTrace();
-      }
-      if (mr != null) {
-        try {
-          mr.shutdown();
-        } catch (Exception ignored) {
-          ignored.printStackTrace();
-        }
-      }
-    }
-  }
-}
-
-/**
- * Dummy socket factory which shift TPC ports by subtracting 10 when
- * establishing a connection
- */
-class DummySocketFactory extends StandardSocketFactory {
-  /**
-   * Default empty constructor (for use with the reflection API).
-   */
-  public DummySocketFactory() {
-  }
-
-  /* @inheritDoc */
-  @Override
-  public Socket createSocket() throws IOException {
-    return new Socket() {
-      @Override
-      public void connect(SocketAddress addr, int timeout)
-          throws IOException {
-
-        assert (addr instanceof InetSocketAddress);
-        InetSocketAddress iaddr = (InetSocketAddress) addr;
-        SocketAddress newAddr = null;
-        if (iaddr.isUnresolved())
-          newAddr =
-              new InetSocketAddress(iaddr.getHostName(),
-                  iaddr.getPort() - 10);
-        else
-          newAddr =
-              new InetSocketAddress(iaddr.getAddress(), iaddr.getPort() - 10);
-        System.out.printf("Test socket: rerouting %s to %s\n", iaddr,
-            newAddr);
-        super.connect(newAddr, timeout);
-      }
-    };
-  }
-
-  /* @inheritDoc */
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (!(obj instanceof DummySocketFactory))
-      return false;
-    return true;
-  }
-
-  /* @inheritDoc */
-  @Override
-  public int hashCode() {
-    // Dummy hash code (to make find bugs happy)
-    return 53;
-  }
-}

+ 0 - 152
src/test/hdfs-with-mr/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java

@@ -1,152 +0,0 @@
-/**
- * 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.security.authorize;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-
-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.hdfs.HDFSPolicyProvider;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.tools.DFSAdmin;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.hadoop.mapred.TestMiniMRWithDFS;
-import org.apache.hadoop.security.UnixUserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
-
-import junit.framework.TestCase;
-
-public class TestServiceLevelAuthorization extends TestCase {
-  public void testServiceLevelAuthorization() throws Exception {
-    MiniDFSCluster dfs = null;
-    MiniMRCluster mr = null;
-    FileSystem fileSys = null;
-    try {
-      final int slaves = 4;
-
-      // Turn on service-level authorization
-      Configuration conf = new Configuration();
-      conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG, 
-                    HadoopPolicyProvider.class, PolicyProvider.class);
-      conf.setBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
-                      true);
-      
-      // Start the mini clusters
-      dfs = new MiniDFSCluster(conf, slaves, true, null);
-      fileSys = dfs.getFileSystem();
-      JobConf mrConf = new JobConf(conf);
-      mr = new MiniMRCluster(slaves, fileSys.getUri().toString(), 1, 
-                             null, null, mrConf);
-
-      // Run examples
-      TestMiniMRWithDFS.runPI(mr, mr.createJobConf(mrConf));
-      TestMiniMRWithDFS.runWordCount(mr, mr.createJobConf(mrConf));
-    } finally {
-      if (dfs != null) { dfs.shutdown(); }
-      if (mr != null) { mr.shutdown();
-      }
-    }
-  }
-  
-  private static final String DUMMY_ACL = "nouser nogroup";
-  private static final String UNKNOWN_USER = "dev,null";
-  
-  private void rewriteHadoopPolicyFile(File policyFile) throws IOException {
-    FileWriter fos = new FileWriter(policyFile);
-    PolicyProvider policyProvider = new HDFSPolicyProvider();
-    fos.write("<configuration>\n");
-    for (Service service : policyProvider.getServices()) {
-      String key = service.getServiceKey();
-      String value ="*";
-      if (key.equals("security.refresh.policy.protocol.acl")) {
-        value = DUMMY_ACL;
-      }
-      fos.write("<property><name>"+ key + "</name><value>" + value + 
-                "</value></property>\n");
-      System.err.println("<property><name>"+ key + "</name><value>" + value + 
-          "</value></property>\n");
-    }
-    fos.write("</configuration>\n");
-    fos.close();
-  }
-  
-  private void refreshPolicy(Configuration conf)  throws IOException {
-    DFSAdmin dfsAdmin = new DFSAdmin(conf);
-    dfsAdmin.refreshServiceAcl();
-  }
-  
-  public void testRefresh() throws Exception {
-    MiniDFSCluster dfs = null;
-    try {
-      final int slaves = 4;
-
-      // Turn on service-level authorization
-      Configuration conf = new Configuration();
-      conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG, 
-                    HDFSPolicyProvider.class, PolicyProvider.class);
-      conf.setBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
-                      true);
-      
-      // Start the mini dfs cluster
-      dfs = new MiniDFSCluster(conf, slaves, true, null);
-
-      // Refresh the service level authorization policy
-      refreshPolicy(conf);
-      
-      // Simulate an 'edit' of hadoop-policy.xml
-      String confDir = System.getProperty("test.build.extraconf", 
-                                          "build/test/extraconf");
-      File policyFile = new File(confDir, ConfiguredPolicy.HADOOP_POLICY_FILE);
-      String policyFileCopy = ConfiguredPolicy.HADOOP_POLICY_FILE + ".orig";
-      FileUtil.copy(policyFile, FileSystem.getLocal(conf),   // first save original 
-                    new Path(confDir, policyFileCopy), false, conf);
-      rewriteHadoopPolicyFile(                               // rewrite the file
-          new File(confDir, ConfiguredPolicy.HADOOP_POLICY_FILE));
-      
-      // Refresh the service level authorization policy
-      refreshPolicy(conf);
-      
-      // Refresh the service level authorization policy once again, 
-      // this time it should fail!
-      try {
-        // Note: hadoop-policy.xml for tests has 
-        // security.refresh.policy.protocol.acl = ${user.name}
-        conf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME, UNKNOWN_USER);
-        refreshPolicy(conf);
-        fail("Refresh of NameNode's policy file cannot be successful!");
-      } catch (RemoteException re) {
-        System.out.println("Good, refresh worked... refresh failed with: " + 
-                           StringUtils.stringifyException(re.unwrapRemoteException()));
-      } finally {
-        // Reset to original hadoop-policy.xml
-        FileUtil.fullyDelete(new File(confDir, 
-            ConfiguredPolicy.HADOOP_POLICY_FILE));
-        FileUtil.replaceFile(new File(confDir, policyFileCopy), new File(confDir, ConfiguredPolicy.HADOOP_POLICY_FILE));
-      }
-    } finally {
-      if (dfs != null) { dfs.shutdown(); }
-    }
-  }
-
-}

+ 0 - 46
src/test/hdfs-with-mr/org/apache/hadoop/test/AllTestDriver.java

@@ -1,46 +0,0 @@
-/**
- * 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.test;
-
-import org.apache.hadoop.util.ProgramDriver;
-
-
-@Deprecated
-//Class to be removed after the project split
-public class AllTestDriver {
-  
-  /**
-   * A description of the test program for running all the tests using jar file
-   */
-  public static void main(String argv[]){
-    ProgramDriver pd = new ProgramDriver();
-    new CoreTestDriver(pd);
-    new HdfsTestDriver(pd);
-    new HdfsWithMRTestDriver(pd);
-    new MapredTestDriver(pd);
-    
-    try {
-      pd.driver(argv);
-    } catch (Throwable e) {
-      e.printStackTrace();
-    }
-  }
-
-}
-

+ 0 - 78
src/test/hdfs-with-mr/org/apache/hadoop/test/HdfsWithMRTestDriver.java

@@ -1,78 +0,0 @@
-/**
- * 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.test;
-
-import org.apache.hadoop.fs.DFSCIOTest;
-import org.apache.hadoop.fs.DistributedFSCheck;
-import org.apache.hadoop.fs.TestDFSIO;
-import org.apache.hadoop.fs.TestFileSystem;
-import org.apache.hadoop.fs.JHLogAnalyzer;
-import org.apache.hadoop.hdfs.NNBench;
-import org.apache.hadoop.io.FileBench;
-import org.apache.hadoop.util.ProgramDriver;
-
-/**
- * Driver for HDFS tests, which require map-reduce to run.
- */
-public class HdfsWithMRTestDriver {
-  
-  
-  private ProgramDriver pgd;
-
-  public HdfsWithMRTestDriver() {
-    this(new ProgramDriver());
-  }
-  
-  public HdfsWithMRTestDriver(ProgramDriver pgd) {
-    this.pgd = pgd;
-    try {
-      pgd.addClass("nnbench", NNBench.class, 
-          "A benchmark that stresses the namenode.");
-      pgd.addClass("testfilesystem", TestFileSystem.class, 
-          "A test for FileSystem read/write.");
-      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.addClass("filebench", FileBench.class, 
-          "Benchmark SequenceFile(Input|Output)Format " +
-          "(block,record compressed and uncompressed), " +
-          "Text(Input|Output)Format (compressed and uncompressed)");
-      pgd.addClass(JHLogAnalyzer.class.getSimpleName(), JHLogAnalyzer.class, 
-          "Job History Log analyzer.");
-    } catch(Throwable e) {
-      e.printStackTrace();
-    }
-  }
-
-  public void run(String argv[]) {
-    try {
-      pgd.driver(argv);
-    } catch(Throwable e) {
-      e.printStackTrace();
-    }
-  }
-
-  public static void main(String argv[]){
-    new HdfsWithMRTestDriver().run(argv);
-  }
-}
-

+ 0 - 13
src/test/mapred-site.xml

@@ -1,13 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!-- Put mapreduce site-specific property overrides in this file. -->
-
-<configuration>
-
-<property>
-  <name>mapred.job.tracker.retire.jobs</name>
-  <value>false</value>
-  <description></description>
-</property>
-</configuration>