瀏覽代碼

MAPREDUCE-6056. native-task: move system test working dir to target dir and cleanup test config xml files (Manu Zhang via bchang)

Binglin Chang 10 年之前
父節點
當前提交
7ade9b04ea
共有 21 個文件被更改,包括 588 次插入580 次删除
  1. 1 0
      hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt
  2. 32 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/CombinerTest.java
  3. 49 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
  4. 18 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/OldAPICombinerTest.java
  5. 15 23
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/compresstest/CompressMapper.java
  6. 62 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/compresstest/CompressTest.java
  7. 15 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/handlers/TestNativeCollectorOnlyHandler.java
  8. 36 47
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
  9. 49 55
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
  10. 20 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/nonsorttest/NonSortTest.java
  11. 30 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/TestConstants.java
  12. 40 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/common_conf.xml
  13. 78 80
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/kvtest-conf.xml
  14. 24 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/native_conf.xml
  15. 21 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/normal_conf.xml
  16. 24 30
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-combiner-conf.xml
  17. 50 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-compress-conf.xml
  18. 0 39
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-gzip-compress-conf.xml
  19. 0 39
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-lz4-compress-conf.xml
  20. 24 25
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-nonsort-conf.xml
  21. 0 41
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-snappy-compress-conf.xml

+ 1 - 0
hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt

@@ -18,3 +18,4 @@ MAPREDUCE-6035. native-task: sources/test-sources jar distribution (Manu Zhang v
 MAPREDUCE-5977. Fix or suppress native-task gcc warnings (Manu Zhang via todd)
 MAPREDUCE-6054. native-task: Speed up tests (todd)
 MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is sucessful (Binglin Chang)
+MAPREDUCE-6056. native-task: move system test working dir to target dir and cleanup test config xml files (Manu Zhang via bchang)

+ 32 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/CombinerTest.java

@@ -38,11 +38,14 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+
 public class CombinerTest {
   private FileSystem fs;
   private String inputpath;
@@ -50,33 +53,25 @@ public class CombinerTest {
   private String hadoopoutputpath;
 
   @Test
-  public void testWordCountCombiner() {
-    try {
-
-      final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
-      nativeConf.addResource(TestConstants.COMBINER_CONF_PATH);
-      final Job nativejob = getJob("nativewordcount", nativeConf, inputpath, nativeoutputpath);
-
-      final Configuration commonConf = ScenarioConfiguration.getNormalConfiguration();
-      commonConf.addResource(TestConstants.COMBINER_CONF_PATH);
-
-      final Job normaljob = getJob("normalwordcount", commonConf, inputpath, hadoopoutputpath);
-
-      assertTrue(nativejob.waitForCompletion(true));
-            
-      Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-      
-      assertTrue(normaljob.waitForCompletion(true));
-      Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-       
-      assertEquals(true, ResultVerifier.verify(nativeoutputpath, hadoopoutputpath));
-      assertEquals("Native Reduce reduce group counter should equal orignal reduce group counter", 
-          nativeReduceGroups.getValue(), normalReduceGroups.getValue());
-      
-    } catch (final Exception e) {
-      e.printStackTrace();
-      assertEquals("run exception", true, false);
-    }
+  public void testWordCountCombiner() throws Exception {
+    final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
+    nativeConf.addResource(TestConstants.COMBINER_CONF_PATH);
+    final Job nativejob = getJob("nativewordcount", nativeConf, inputpath, nativeoutputpath);
+
+    final Configuration commonConf = ScenarioConfiguration.getNormalConfiguration();
+    commonConf.addResource(TestConstants.COMBINER_CONF_PATH);
+    final Job normaljob = getJob("normalwordcount", commonConf, inputpath, hadoopoutputpath);
+
+    assertTrue(nativejob.waitForCompletion(true));
+
+    Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
+
+    assertTrue(normaljob.waitForCompletion(true));
+    Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
+
+    assertEquals(true, ResultVerifier.verify(nativeoutputpath, hadoopoutputpath));
+    assertEquals("Native Reduce reduce group counter should equal orignal reduce group counter",
+      nativeReduceGroups.getValue(), normalReduceGroups.getValue());
   }
 
   @Before
@@ -88,8 +83,7 @@ public class CombinerTest {
 
     this.fs = FileSystem.get(conf);
 
-    this.inputpath = conf.get(TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_KEY,
-        TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_DEFAULTV) + "/wordcount";
+    this.inputpath = TestConstants.NATIVETASK_COMBINER_TEST_INPUTDIR + "/wordcount";
 
     if (!fs.exists(new Path(inputpath))) {
       new TestInputFile(
@@ -98,10 +92,15 @@ public class CombinerTest {
           Text.class.getName(), conf).createSequenceTestFile(inputpath, 1, (byte)('a'));
     }
 
-    this.nativeoutputpath = conf.get(TestConstants.NATIVETASK_TEST_COMBINER_OUTPUTPATH,
-        TestConstants.NATIVETASK_TEST_COMBINER_OUTPUTPATH_DEFAULTV) + "/nativewordcount";
-    this.hadoopoutputpath = conf.get(TestConstants.NORMAL_TEST_COMBINER_OUTPUTPATH,
-        TestConstants.NORMAL_TEST_COMBINER_OUTPUTPATH_DEFAULTV) + "/normalwordcount";
+    this.nativeoutputpath = TestConstants.NATIVETASK_COMBINER_TEST_NATIVE_OUTPUTDIR + "/nativewordcount";
+    this.hadoopoutputpath = TestConstants.NATIVETASK_COMBINER_TEST_NORMAL_OUTPUTDIR + "/normalwordcount";
+  }
+
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_COMBINER_TEST_DIR), true);
+    fs.close();
   }
 
   protected static Job getJob(String jobname, Configuration inputConf, String inputpath, String outputpath)

+ 49 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java

@@ -35,11 +35,14 @@ import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Job;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+
 public class LargeKVCombinerTest {
   private static final Log LOG = LogFactory.getLog(LargeKVCombinerTest.class);
 
@@ -50,61 +53,62 @@ public class LargeKVCombinerTest {
   }
 
   @Test
-  public void testLargeValueCombiner(){
+  public void testLargeValueCombiner() throws Exception {
     final Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
     final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
     normalConf.addResource(TestConstants.COMBINER_CONF_PATH);
     nativeConf.addResource(TestConstants.COMBINER_CONF_PATH);
     final int deafult_KVSize_Maximum = 1 << 22; // 4M
-    final int KVSize_Maximu = normalConf.getInt(TestConstants.NATIVETASK_KVSIZE_MAX_LARGEKV_TEST,
+    final int KVSize_Maximum = normalConf.getInt(TestConstants.NATIVETASK_KVSIZE_MAX_LARGEKV_TEST,
         deafult_KVSize_Maximum);
-    final String inputPath = normalConf.get(TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_KEY,
-        TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_DEFAULTV) + "/largeKV";
-    final String nativeOutputPath = normalConf.get(TestConstants.NATIVETASK_TEST_COMBINER_OUTPUTPATH,
-        TestConstants.NATIVETASK_TEST_COMBINER_OUTPUTPATH_DEFAULTV) + "/nativeLargeKV";
-    final String hadoopOutputPath = normalConf.get(TestConstants.NORMAL_TEST_COMBINER_OUTPUTPATH,
-        TestConstants.NORMAL_TEST_COMBINER_OUTPUTPATH_DEFAULTV) + "/normalLargeKV";
-    try {
-      final FileSystem fs = FileSystem.get(normalConf);
-      for (int i = 65536; i <= KVSize_Maximu; i *= 4) {
-        
-        int max = i;
-        int min = Math.max(i / 4, max - 10);
-        
-        LOG.info("===KV Size Test: min size: " + min + ", max size: " + max);
-        
-        normalConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
-        normalConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
-        nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
-        nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
-        fs.delete(new Path(inputPath), true);
-        new TestInputFile(normalConf.getInt(TestConstants.NATIVETASK_COMBINER_WORDCOUNT_FILESIZE,
-            1000000), IntWritable.class.getName(),
-            Text.class.getName(), normalConf).createSequenceTestFile(inputPath, 1);
-        
-        final Job normaljob = CombinerTest.getJob("normalwordcount", normalConf, inputPath, hadoopOutputPath);
-        final Job nativejob = CombinerTest.getJob("nativewordcount", nativeConf, inputPath, nativeOutputPath);
-        
-        assertTrue(nativejob.waitForCompletion(true));
+    final String inputPath = TestConstants.NATIVETASK_COMBINER_TEST_INPUTDIR + "/largeKV";
+    final String nativeOutputPath = TestConstants.NATIVETASK_COMBINER_TEST_NATIVE_OUTPUTDIR
+      + "/nativeLargeKV";
+    final String hadoopOutputPath = TestConstants.NATIVETASK_COMBINER_TEST_NORMAL_OUTPUTDIR
+      + "/normalLargeKV";
+    final FileSystem fs = FileSystem.get(normalConf);
+    for (int i = 65536; i <= KVSize_Maximum; i *= 4) {
+
+      int max = i;
+      int min = Math.max(i / 4, max - 10);
+
+      LOG.info("===KV Size Test: min size: " + min + ", max size: " + max);
+
+      normalConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
+      normalConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
+      nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
+      nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
+      fs.delete(new Path(inputPath), true);
+      new TestInputFile(normalConf.getInt(TestConstants.NATIVETASK_COMBINER_WORDCOUNT_FILESIZE,
+        1000000), IntWritable.class.getName(),
+        Text.class.getName(), normalConf).createSequenceTestFile(inputPath, 1);
+
+      final Job normaljob = CombinerTest.getJob("normalwordcount", normalConf, inputPath, hadoopOutputPath);
+      final Job nativejob = CombinerTest.getJob("nativewordcount", nativeConf, inputPath, nativeOutputPath);
+
+      assertTrue(nativejob.waitForCompletion(true));
         Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-        
-        assertTrue(normaljob.waitForCompletion(true));
+
+      assertTrue(normaljob.waitForCompletion(true));
         Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-        
-        final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
-        
-        final String reason = "LargeKVCombinerTest failed with, min size: " + min
-            + ", max size: " + max + ", normal out: " + hadoopOutputPath + ", native Out: " + nativeOutputPath;
-        
-        assertEquals(reason, true, compareRet);
-//        assertEquals("Native Reduce reduce group counter should equal orignal reduce group counter", 
+
+      final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
+
+      final String reason = "LargeKVCombinerTest failed with, min size: " + min
+        + ", max size: " + max + ", normal out: " + hadoopOutputPath + ", native Out: " + nativeOutputPath;
+
+      assertEquals(reason, true, compareRet);
+//        assertEquals("Native Reduce reduce group counter should equal orignal reduce group counter",
 //            nativeReduceGroups.getValue(), normalReduceGroups.getValue());
-      }
-      fs.close();
-    } catch (final Exception e) {
-      e.printStackTrace();
-      assertEquals("run exception", true, false);
     }
+    fs.close();
+  }
+
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_COMBINER_TEST_DIR), true);
+    fs.close();
   }
   
 }

+ 18 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/OldAPICombinerTest.java

@@ -38,11 +38,14 @@ import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Counter;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+
 public class OldAPICombinerTest {
   private FileSystem fs;
   private String inputpath;
@@ -51,23 +54,23 @@ public class OldAPICombinerTest {
   public void testWordCountCombinerWithOldAPI() throws Exception {
     final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
     nativeConf.addResource(TestConstants.COMBINER_CONF_PATH);
-    final String nativeoutput = nativeConf.get(TestConstants.OLDAPI_NATIVETASK_TEST_COMBINER_OUTPUTPATH);
+    final String nativeoutput = TestConstants.NATIVETASK_OLDAPI_COMBINER_TEST_NATIVE_OUTPUTPATH;
     final JobConf nativeJob = getOldAPIJobconf(nativeConf, "nativeCombinerWithOldAPI", inputpath, nativeoutput);
     RunningJob nativeRunning = JobClient.runJob(nativeJob);
 
     Counter nativeReduceGroups = nativeRunning.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-    
+
     final Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
     normalConf.addResource(TestConstants.COMBINER_CONF_PATH);
-    final String normaloutput = normalConf.get(TestConstants.OLDAPI_NORMAL_TEST_COMBINER_OUTPUTPATH);
+    final String normaloutput = TestConstants.NATIVETASK_OLDAPI_COMBINER_TEST_NORMAL_OUTPUTPATH;
     final JobConf normalJob = getOldAPIJobconf(normalConf, "normalCombinerWithOldAPI", inputpath, normaloutput);
-    
+
     RunningJob normalRunning = JobClient.runJob(normalJob);
     Counter normalReduceGroups = normalRunning.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
-    
+
     final boolean compareRet = ResultVerifier.verify(nativeoutput, normaloutput);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
-    
+
     assertEquals("The input reduce record count must be same", nativeReduceGroups.getValue(), normalReduceGroups.getValue());
   }
 
@@ -78,15 +81,21 @@ public class OldAPICombinerTest {
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     conf.addcombinerConf();
     this.fs = FileSystem.get(conf);
-    this.inputpath = conf.get(TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_KEY,
-        TestConstants.NATIVETASK_TEST_COMBINER_INPUTPATH_DEFAULTV) + "/wordcount";
+    this.inputpath = TestConstants.NATIVETASK_COMBINER_TEST_INPUTDIR + "/wordcount";
 
     if (!fs.exists(new Path(inputpath))) {
-      new TestInputFile(conf.getInt("nativetask.combiner.wordcount.filesize", 1000000), Text.class.getName(),
+      new TestInputFile(conf.getInt(TestConstants.NATIVETASK_COMBINER_WORDCOUNT_FILESIZE, 1000000), Text.class.getName(),
           Text.class.getName(), conf).createSequenceTestFile(inputpath, 1, (byte)('a'));
     }
   }
 
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_COMBINER_TEST_DIR), true);
+    fs.close();
+  }
+
   private static JobConf getOldAPIJobconf(Configuration configuration, String name, String input, String output)
       throws Exception {
     final JobConf jobConf = new JobConf(configuration);

+ 15 - 23
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/compresstest/CompressMapper.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
-import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -32,8 +31,6 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
 public class CompressMapper {
-  public static final String inputFile = "./compress/input.txt";
-  public static final String outputFileDir = "./compress/output/";
 
   public static class TextCompressMapper extends Mapper<Text, Text, Text, Text> {
 
@@ -43,27 +40,22 @@ public class CompressMapper {
     }
   }
 
-  public static Job getCompressJob(String jobname, Configuration conf) {
-    Job job = null;
-    try {
-      job = new Job(conf, jobname + "-CompressMapperJob");
-      job.setJarByClass(CompressMapper.class);
-      job.setMapperClass(TextCompressMapper.class);
-      job.setOutputKeyClass(Text.class);
-      job.setMapOutputValueClass(Text.class);
-      final Path outputpath = new Path(outputFileDir + jobname);
-      // if output file exists ,delete it
-      final FileSystem hdfs = FileSystem.get(new ScenarioConfiguration());
-      if (hdfs.exists(outputpath)) {
-        hdfs.delete(outputpath);
-      }
-      hdfs.close();
-      job.setInputFormatClass(SequenceFileInputFormat.class);
-      FileInputFormat.addInputPath(job, new Path(inputFile));
-      FileOutputFormat.setOutputPath(job, outputpath);
-    } catch (final Exception e) {
-      e.printStackTrace();
+  public static Job getCompressJob(String jobname, Configuration conf, String inputpath, String outputpath)
+    throws Exception {
+    Job job = new Job(conf, jobname + "-CompressMapperJob");
+    job.setJarByClass(CompressMapper.class);
+    job.setMapperClass(TextCompressMapper.class);
+    job.setOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
+    // if output file exists ,delete it
+    final FileSystem hdfs = FileSystem.get(new ScenarioConfiguration());
+    if (hdfs.exists(new Path(outputpath))) {
+      hdfs.delete(new Path(outputpath));
     }
+    hdfs.close();
+    job.setInputFormatClass(SequenceFileInputFormat.class);
+    FileInputFormat.addInputPath(job, new Path(inputpath));
+    FileOutputFormat.setOutputPath(job, new Path(outputpath));
     return job;
   }
 }

+ 62 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/compresstest/CompressTest.java

@@ -30,60 +30,87 @@ import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+
 public class CompressTest {
 
+  private static final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
+  private static final Configuration hadoopConf = ScenarioConfiguration.getNormalConfiguration();
+
+  static {
+    nativeConf.addResource(TestConstants.COMPRESS_TEST_CONF_PATH);
+    hadoopConf.addResource(TestConstants.COMPRESS_TEST_CONF_PATH);
+  }
+
   @Test
   public void testSnappyCompress() throws Exception {
-    final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
-    conf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
-    final Job job = CompressMapper.getCompressJob("nativesnappy", conf);
+    final String snappyCodec = "org.apache.hadoop.io.compress.SnappyCodec";
+
+    nativeConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, snappyCodec);
+    final String nativeOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NATIVE_OUTPUTDIR + "/snappy";
+    final Job job = CompressMapper.getCompressJob("nativesnappy", nativeConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, nativeOutputPath);
     assertTrue(job.waitForCompletion(true));
 
-    final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
-    hadoopconf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
-    final Job hadoopjob = CompressMapper.getCompressJob("hadoopsnappy", hadoopconf);
+    hadoopConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, snappyCodec);
+    final String hadoopOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NORMAL_OUTPUTDIR + "/snappy";
+    final Job hadoopjob = CompressMapper.getCompressJob("hadoopsnappy", hadoopConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, hadoopOutputPath);
     assertTrue(hadoopjob.waitForCompletion(true));
 
-    final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativesnappy",
-        CompressMapper.outputFileDir + "hadoopsnappy");
+    final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
   }
 
   @Test
   public void testGzipCompress() throws Exception {
-    final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
-    conf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
-    final Job job = CompressMapper.getCompressJob("nativegzip", conf);
+    final String gzipCodec = "org.apache.hadoop.io.compress.GzipCodec";
+
+    nativeConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, gzipCodec);
+    final String nativeOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NATIVE_OUTPUTDIR + "/gzip";
+    final Job job = CompressMapper.getCompressJob("nativegzip", nativeConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, nativeOutputPath);
     assertTrue(job.waitForCompletion(true));
 
-    final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
-    hadoopconf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
-    final Job hadoopjob = CompressMapper.getCompressJob("hadoopgzip", hadoopconf);
+    hadoopConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, gzipCodec);
+    final String hadoopOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NORMAL_OUTPUTDIR + "/gzip";
+    final Job hadoopjob = CompressMapper.getCompressJob("hadoopgzip", hadoopConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, hadoopOutputPath);
     assertTrue(hadoopjob.waitForCompletion(true));
 
-    final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativegzip",
-        CompressMapper.outputFileDir + "hadoopgzip");
+    final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
   }
 
   @Test
   public void testLz4Compress() throws Exception {
-    final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
-    nativeConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
-    final Job nativeJob = CompressMapper.getCompressJob("nativelz4", nativeConf);
+    final String lz4Codec = "org.apache.hadoop.io.compress.Lz4Codec";
+
+    nativeConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, lz4Codec);
+    final String nativeOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NATIVE_OUTPUTDIR + "/lz4";
+    final Job nativeJob = CompressMapper.getCompressJob("nativelz4", nativeConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, nativeOutputPath);
     assertTrue(nativeJob.waitForCompletion(true));
 
-    final Configuration hadoopConf = ScenarioConfiguration.getNormalConfiguration();
-    hadoopConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
-    final Job hadoopJob = CompressMapper.getCompressJob("hadooplz4", hadoopConf);
+    hadoopConf.set(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, lz4Codec);
+    final String hadoopOutputPath =
+      TestConstants.NATIVETASK_COMPRESS_TEST_NORMAL_OUTPUTDIR + "/lz4";
+    final Job hadoopJob = CompressMapper.getCompressJob("hadooplz4", hadoopConf,
+      TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR, hadoopOutputPath);
     assertTrue(hadoopJob.waitForCompletion(true));
-    final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativelz4",
-        CompressMapper.outputFileDir + "hadooplz4");
+    final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
   }
 
@@ -93,15 +120,22 @@ public class CompressTest {
     Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     final FileSystem fs = FileSystem.get(conf);
-    final Path path = new Path(CompressMapper.inputFile);
+    final Path path = new Path(TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR);
     fs.delete(path);
     if (!fs.exists(path)) {
-      new TestInputFile(ScenarioConfiguration.getNormalConfiguration().getInt(
+      new TestInputFile(hadoopConf.getInt(
           TestConstants.NATIVETASK_COMPRESS_FILESIZE, 100000),
           Text.class.getName(), Text.class.getName(), conf)
-      .createSequenceTestFile(CompressMapper.inputFile);
-
+      .createSequenceTestFile(TestConstants.NATIVETASK_COMPRESS_TEST_INPUTDIR);
     }
     fs.close();
   }
+
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_COMPRESS_TEST_DIR), true);
+    fs.close();
+  }
+
 }

+ 15 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/handlers/TestNativeCollectorOnlyHandler.java

@@ -22,6 +22,9 @@ import java.io.IOException;
 import junit.framework.Assert;
 import junit.framework.TestCase;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.nativetask.Command;
@@ -30,6 +33,7 @@ import org.apache.hadoop.mapred.nativetask.INativeHandler;
 import org.apache.hadoop.mapred.nativetask.TaskContext;
 import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
 import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
+import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.util.OutputUtil;
 import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
 import org.mockito.Matchers;
@@ -43,7 +47,7 @@ public class TestNativeCollectorOnlyHandler extends TestCase {
   private BufferPusher pusher;
   private ICombineHandler combiner;
   private TaskContext taskContext;
-  private String localDir = "build/test/mapred/local";
+  private static final String LOCAL_DIR = TestConstants.NATIVETASK_TEST_DIR + "/local";
 
   @Override
   public void setUp() throws IOException {
@@ -53,7 +57,7 @@ public class TestNativeCollectorOnlyHandler extends TestCase {
     JobConf jobConf = new JobConf();
     jobConf.set(OutputUtil.NATIVE_TASK_OUTPUT_MANAGER,
         "org.apache.hadoop.mapred.nativetask.util.LocalJobOutputFiles");
-    jobConf.set("mapred.local.dir", localDir);
+    jobConf.set("mapred.local.dir", LOCAL_DIR);
     this.taskContext = new TaskContext(jobConf,
         BytesWritable.class, BytesWritable.class,
         BytesWritable.class,
@@ -64,6 +68,12 @@ public class TestNativeCollectorOnlyHandler extends TestCase {
     Mockito.when(nativeHandler.getInputBuffer()).thenReturn(new InputBuffer(BufferType.HEAP_BUFFER, 100));
   }
 
+  @Override
+  public void tearDown() throws IOException {
+    FileSystem.getLocal(new Configuration()).delete(new Path(LOCAL_DIR));
+  }
+
+
   public void testCollect() throws IOException {
     this.handler = new NativeCollectorOnlyHandler(taskContext, nativeHandler, pusher, combiner);
     handler.collect(new BytesWritable(), new BytesWritable(), 100);
@@ -95,9 +105,9 @@ public class TestNativeCollectorOnlyHandler extends TestCase {
     }
     Assert.assertTrue("exception thrown", thrown);
 
-    final String expectedOutputPath = localDir + "/output/file.out";
-    final String expectedOutputIndexPath = localDir + "/output/file.out.index";
-    final String expectedSpillPath = localDir + "/output/spill0.out";
+    final String expectedOutputPath = LOCAL_DIR + "/output/file.out";
+    final String expectedOutputIndexPath = LOCAL_DIR + "/output/file.out.index";
+    final String expectedSpillPath = LOCAL_DIR + "/output/spill0.out";
 
     final String outputPath = handler.onCall(NativeCollectorOnlyHandler.GET_OUTPUT_PATH, null).readString();
     Assert.assertEquals(expectedOutputPath, outputPath);

+ 36 - 47
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
@@ -112,70 +113,58 @@ public class KVTest {
   }
 
   @Test
-  public void testKVCompability() {
-    try {
-      final String nativeoutput = this.runNativeTest(
-          "Test:" + keyclass.getSimpleName() + "--" + valueclass.getSimpleName(), keyclass, valueclass);
-      final String normaloutput = this.runNormalTest(
-          "Test:" + keyclass.getSimpleName() + "--" + valueclass.getSimpleName(), keyclass, valueclass);
-      final boolean compareRet = ResultVerifier.verify(normaloutput, nativeoutput);
-      final String input = nativekvtestconf.get(TestConstants.NATIVETASK_KVTEST_INPUTDIR) + "/"
-          + keyclass.getName()
-          + "/" + valueclass.getName();
-      if(compareRet){
-        final FileSystem fs = FileSystem.get(hadoopkvtestconf);
-        fs.delete(new Path(nativeoutput), true);
-        fs.delete(new Path(normaloutput), true);
-        fs.delete(new Path(input), true);
-        fs.close();
-      }
-      assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
-    } catch (final IOException e) {
-      assertEquals("test run exception:", null, e);
-    } catch (final Exception e) {
-      assertEquals("test run exception:", null, e);
+  public void testKVCompability() throws Exception {
+    final String nativeoutput = this.runNativeTest(
+      "Test:" + keyclass.getSimpleName() + "--" + valueclass.getSimpleName(), keyclass, valueclass);
+    final String normaloutput = this.runNormalTest(
+      "Test:" + keyclass.getSimpleName() + "--" + valueclass.getSimpleName(), keyclass, valueclass);
+    final boolean compareRet = ResultVerifier.verify(normaloutput, nativeoutput);
+    final String input = TestConstants.NATIVETASK_KVTEST_INPUTDIR + "/"
+      + keyclass.getName() + "/" + valueclass.getName();
+    if(compareRet){
+      final FileSystem fs = FileSystem.get(hadoopkvtestconf);
+      fs.delete(new Path(nativeoutput), true);
+      fs.delete(new Path(normaloutput), true);
+      fs.delete(new Path(input), true);
+      fs.close();
     }
+    assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
   }
 
-  private String runNativeTest(String jobname, Class<?> keyclass, Class<?> valueclass) throws IOException {
-    final String inputpath = nativekvtestconf.get(TestConstants.NATIVETASK_KVTEST_INPUTDIR) + "/"
-        + keyclass.getName()
-        + "/" + valueclass.getName();
-    final String outputpath = nativekvtestconf.get(TestConstants.NATIVETASK_KVTEST_OUTPUTDIR) + "/"
-        + keyclass.getName() + "/" + valueclass.getName();
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_KVTEST_DIR), true);
+    fs.close();
+  }
+
+  private String runNativeTest(String jobname, Class<?> keyclass, Class<?> valueclass) throws Exception {
+    final String inputpath = TestConstants.NATIVETASK_KVTEST_INPUTDIR + "/"
+      + keyclass.getName() + "/" + valueclass.getName();
+    final String outputpath = TestConstants.NATIVETASK_KVTEST_NATIVE_OUTPUTDIR + "/"
+      + keyclass.getName() + "/" + valueclass.getName();
     // if output file exists ,then delete it
     final FileSystem fs = FileSystem.get(nativekvtestconf);
     fs.delete(new Path(outputpath));
     fs.close();
     nativekvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "true");
-    try {
-      final KVJob keyJob = new KVJob(jobname, nativekvtestconf, keyclass, valueclass, inputpath, outputpath);
-      assertTrue("job should complete successfully", keyJob.runJob());
-    } catch (final Exception e) {
-      return "native testcase run time error.";
-    }
+    final KVJob keyJob = new KVJob(jobname, nativekvtestconf, keyclass, valueclass, inputpath, outputpath);
+    assertTrue("job should complete successfully", keyJob.runJob());
     return outputpath;
   }
 
-  private String runNormalTest(String jobname, Class<?> keyclass, Class<?> valueclass) throws IOException {
-    final String inputpath = hadoopkvtestconf.get(TestConstants.NATIVETASK_KVTEST_INPUTDIR) + "/"
-        + keyclass.getName()
-        + "/" + valueclass.getName();
-    final String outputpath = hadoopkvtestconf
-        .get(TestConstants.NATIVETASK_KVTEST_NORMAL_OUTPUTDIR)
-        + "/"
+  private String runNormalTest(String jobname, Class<?> keyclass, Class<?> valueclass) throws Exception {
+    final String inputpath = TestConstants.NATIVETASK_KVTEST_INPUTDIR + "/"
+        + keyclass.getName() + "/" + valueclass.getName();
+    final String outputpath = TestConstants.NATIVETASK_KVTEST_NORMAL_OUTPUTDIR + "/"
         + keyclass.getName() + "/" + valueclass.getName();
     // if output file exists ,then delete it
     final FileSystem fs = FileSystem.get(hadoopkvtestconf);
     fs.delete(new Path(outputpath));
     fs.close();
     hadoopkvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "false");
-    try {
-      final KVJob keyJob = new KVJob(jobname, hadoopkvtestconf, keyclass, valueclass, inputpath, outputpath);
-      assertTrue("job should complete successfully", keyJob.runJob());
-    } catch (final Exception e) {
-      return "normal testcase run time error.";
-    }
+    final KVJob keyJob = new KVJob(jobname, hadoopkvtestconf, keyclass, valueclass, inputpath, outputpath);
+    assertTrue("job should complete successfully", keyJob.runJob());
     return outputpath;
   }
 

+ 49 - 55
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
@@ -47,97 +48,90 @@ public class LargeKVTest {
     Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
   }
 
+  private static Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
+  private static Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
+  static {
+    nativeConf.addResource(TestConstants.KVTEST_CONF_PATH);
+    nativeConf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "true");
+    normalConf.addResource(TestConstants.KVTEST_CONF_PATH);
+    normalConf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "false");
+  }
+
   @Test
-  public void testKeySize() {
+  public void testKeySize() throws Exception {
     runKVSizeTests(Text.class, IntWritable.class);
   }
 
   @Test
-  public void testValueSize() {
+  public void testValueSize() throws Exception {
     runKVSizeTests(IntWritable.class, Text.class);
   }
 
-  private static Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
-  private static Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
-  static {
-    nativeConf.addResource(TestConstants.KVTEST_CONF_PATH);
-    nativeConf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "true");
-    normalConf.addResource(TestConstants.KVTEST_CONF_PATH);
-    normalConf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "false");
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_KVTEST_DIR), true);
+    fs.close();
   }
 
-  public void runKVSizeTests(Class<?> keyClass, Class<?> valueClass) {
+  public void runKVSizeTests(Class<?> keyClass, Class<?> valueClass) throws Exception {
     if (!keyClass.equals(Text.class) && !valueClass.equals(Text.class)) {
       return;
     }
     final int deafult_KVSize_Maximum = 1 << 22; // 4M
     final int KVSize_Maximu = normalConf.getInt(TestConstants.NATIVETASK_KVSIZE_MAX_LARGEKV_TEST,
         deafult_KVSize_Maximum);
-    try {
 
-      for (int i = 65536; i <= KVSize_Maximu; i *= 4) {
-        int min = i / 4;
-        int max = i;
-        nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
-        nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
-        normalConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
-        normalConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
+    for (int i = 65536; i <= KVSize_Maximu; i *= 4) {
+      int min = i / 4;
+      int max = i;
+      nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
+      nativeConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
+      normalConf.set(TestConstants.NATIVETASK_KVSIZE_MIN, String.valueOf(min));
+      normalConf.set(TestConstants.NATIVETASK_KVSIZE_MAX, String.valueOf(max));
 
-        LOG.info("===KV Size Test: min size: " + min + ", max size: " + max + ", keyClass: "
-          + keyClass.getName() + ", valueClass: " + valueClass.getName());
+      LOG.info("===KV Size Test: min size: " + min + ", max size: " + max + ", keyClass: "
+        + keyClass.getName() + ", valueClass: " + valueClass.getName());
 
-        final String nativeOutPut = runNativeLargeKVTest("Test Large Value Size:" + String.valueOf(i), keyClass,
-            valueClass, nativeConf);
-        final String normalOutPut = this.runNormalLargeKVTest("Test Large Key Size:" + String.valueOf(i), keyClass,
-            valueClass, normalConf);
-        final boolean compareRet = ResultVerifier.verify(normalOutPut, nativeOutPut);
-        final String reason = "keytype: " + keyClass.getName() + ", valuetype: " + valueClass.getName()
-            + ", failed with " + (keyClass.equals(Text.class) ? "key" : "value") + ", min size: " + min
-            + ", max size: " + max + ", normal out: " + normalOutPut + ", native Out: " + nativeOutPut;
-        assertEquals(reason, true, compareRet);
-      }
-    } catch (final Exception e) {
-      // TODO: handle exception
-      // assertEquals("test run exception:", null, e);
-      e.printStackTrace();
+      final String nativeOutPut = runNativeLargeKVTest("Test Large Value Size:" + String.valueOf(i), keyClass,
+        valueClass, nativeConf);
+      final String normalOutPut = this.runNormalLargeKVTest("Test Large Key Size:" + String.valueOf(i), keyClass,
+        valueClass, normalConf);
+      final boolean compareRet = ResultVerifier.verify(normalOutPut, nativeOutPut);
+      final String reason = "keytype: " + keyClass.getName() + ", valuetype: " + valueClass.getName()
+        + ", failed with " + (keyClass.equals(Text.class) ? "key" : "value") + ", min size: " + min
+        + ", max size: " + max + ", normal out: " + normalOutPut + ", native Out: " + nativeOutPut;
+      assertEquals(reason, true, compareRet);
     }
   }
 
   private String runNativeLargeKVTest(String jobname, Class<?> keyclass, Class<?> valueclass, Configuration conf)
       throws Exception {
-    final String inputpath = conf.get(TestConstants.NATIVETASK_KVTEST_INPUTDIR) + "/LargeKV/" + keyclass.getName()
-        + "/" + valueclass.getName();
-    final String outputpath = conf.get(TestConstants.NATIVETASK_KVTEST_OUTPUTDIR) + "/LargeKV/" + keyclass.getName()
-        + "/" + valueclass.getName();
+    final String inputpath = TestConstants.NATIVETASK_KVTEST_INPUTDIR + "/LargeKV/"
+      + keyclass.getName() + "/" + valueclass.getName();
+    final String outputpath = TestConstants.NATIVETASK_KVTEST_NATIVE_OUTPUTDIR + "/LargeKV/"
+      + keyclass.getName() + "/" + valueclass.getName();
     // if output file exists ,then delete it
     final FileSystem fs = FileSystem.get(conf);
     fs.delete(new Path(outputpath), true);
     fs.close();
-    try {
-      final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
-      assertTrue("job should complete successfully", keyJob.runJob());
-    } catch (final Exception e) {
-      return "normal testcase run time error.";
-    }
+    final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
+    assertTrue("job should complete successfully", keyJob.runJob());
     return outputpath;
   }
 
   private String runNormalLargeKVTest(String jobname, Class<?> keyclass, Class<?> valueclass, Configuration conf)
-      throws IOException {
-    final String inputpath = conf.get(TestConstants.NATIVETASK_KVTEST_INPUTDIR) + "/LargeKV/" + keyclass.getName()
-        + "/" + valueclass.getName();
-    final String outputpath = conf.get(TestConstants.NATIVETASK_KVTEST_NORMAL_OUTPUTDIR) + "/LargeKV/"
-        + keyclass.getName() + "/" + valueclass.getName();
+      throws Exception {
+    final String inputpath = TestConstants.NATIVETASK_KVTEST_INPUTDIR + "/LargeKV/"
+      + keyclass.getName() + "/" + valueclass.getName();
+    final String outputpath = TestConstants.NATIVETASK_KVTEST_NORMAL_OUTPUTDIR + "/LargeKV/"
+      + keyclass.getName() + "/" + valueclass.getName();
     // if output file exists ,then delete it
     final FileSystem fs = FileSystem.get(conf);
     fs.delete(new Path(outputpath), true);
     fs.close();
-    try {
-      final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
-      assertTrue("job should complete successfully", keyJob.runJob());
-    } catch (final Exception e) {
-      return "normal testcase run time error.";
-    }
+    final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
+    assertTrue("job should complete successfully", keyJob.runJob());
     return outputpath;
   }
 }

+ 20 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/nonsorttest/NonSortTest.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.junit.AfterClass;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
@@ -49,20 +50,21 @@ public class NonSortTest {
     Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
     nativeConf.addResource(TestConstants.NONSORT_TEST_CONF);
     nativeConf.set(TestConstants.NATIVETASK_MAP_OUTPUT_SORT, "false");
-    String inputpath = nativeConf.get(TestConstants.NONSORT_TEST_INPUTDIR);
-    String outputpath = nativeConf.get(TestConstants.NONSORT_TEST_NATIVE_OUTPUT);
-    final Job nativeNonSort = getJob(nativeConf, "NativeNonSort", inputpath, outputpath);
+    final Job nativeNonSort = getJob(nativeConf, "NativeNonSort",
+      TestConstants.NATIVETASK_NONSORT_TEST_INPUTDIR,
+      TestConstants.NATIVETASK_NONSORT_TEST_NATIVE_OUTPUT);
     nativeNonSort.waitForCompletion(true);
 
     Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
     normalConf.addResource(TestConstants.NONSORT_TEST_CONF);
-    inputpath = normalConf.get(TestConstants.NONSORT_TEST_INPUTDIR);
-    outputpath = normalConf.get(TestConstants.NONSORT_TEST_NORMAL_OUTPUT);
-    final Job hadoopWithSort = getJob(normalConf, "NormalJob", inputpath, outputpath);
+    final Job hadoopWithSort = getJob(normalConf, "NormalJob",
+      TestConstants.NATIVETASK_NONSORT_TEST_INPUTDIR,
+      TestConstants.NATIVETASK_NONSORT_TEST_NORMAL_OUTPUT);
     hadoopWithSort.waitForCompletion(true);
 
-    final boolean compareRet = ResultVerifier.verify(nativeConf.get(TestConstants.NONSORT_TEST_NATIVE_OUTPUT),
-        normalConf.get(TestConstants.NONSORT_TEST_NORMAL_OUTPUT));
+    final boolean compareRet = ResultVerifier.verify(
+      TestConstants.NATIVETASK_NONSORT_TEST_NATIVE_OUTPUT,
+      TestConstants.NATIVETASK_NONSORT_TEST_NORMAL_OUTPUT);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
   }
 
@@ -73,14 +75,22 @@ public class NonSortTest {
     final ScenarioConfiguration configuration = new ScenarioConfiguration();
     configuration.addNonSortTestConf();
     final FileSystem fs = FileSystem.get(configuration);
-    final Path path = new Path(configuration.get(TestConstants.NONSORT_TEST_INPUTDIR));
+    final Path path = new Path(TestConstants.NATIVETASK_NONSORT_TEST_INPUTDIR);
     if (!fs.exists(path)) {
-      new TestInputFile(configuration.getInt("nativetask.nonsorttest.filesize", 10000000), Text.class.getName(),
+      new TestInputFile(configuration.getInt(TestConstants.NATIVETASK_NONSORTTEST_FILESIZE, 10000000), Text.class.getName(),
           Text.class.getName(), configuration).createSequenceTestFile(path.toString());
     }
     fs.close();
   }
 
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    final FileSystem fs = FileSystem.get(new ScenarioConfiguration());
+    fs.delete(new Path(TestConstants.NATIVETASK_NONSORT_TEST_DIR), true);
+    fs.close();
+  }
+
+
   private Job getJob(Configuration conf, String jobName, String inputpath, String outputpath) throws IOException {
     final FileSystem fs = FileSystem.get(conf);
     if (fs.exists(new Path(outputpath))) {

+ 30 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/TestConstants.java

@@ -19,49 +19,53 @@ package org.apache.hadoop.mapred.nativetask.testutil;
 
 public class TestConstants {
   // conf path
+  public static final String COMMON_CONF_PATH = "common_conf.xml";
   public static final String COMBINER_CONF_PATH = "test-combiner-conf.xml";
   public static final String KVTEST_CONF_PATH = "kvtest-conf.xml";
   public static final String NONSORT_TEST_CONF = "test-nonsort-conf.xml";
+  public static final String COMPRESS_TEST_CONF_PATH = "test-compress-conf.xml";
 
+  // common constants
+  public static final String NATIVETASK_TEST_DIR = System.getProperty("test.build.data", "target/test/data");
+  public static final String NATIVETASK_COLLECTOR_DELEGATOR = "mapreduce.map.output.collector.delegator.class";
+  public static final String NATIVETASK_COLLECTOR_DELEGATOR_CLASS = "org.apache.hadoop.mapred.nativetask.testutil.EnforceNativeOutputCollectorDelegator";
   public static final String NATIVETASK_KVSIZE_MIN = "nativetask.kvsize.min";
   public static final String NATIVETASK_KVSIZE_MAX = "nativetask.kvsize.max";
+  public static final String NATIVETASK_KVSIZE_MAX_LARGEKV_TEST = "nativetask.kvsize.max.largekv";
 
-  public static final String NATIVETASK_KVTEST_INPUTDIR = "nativetask.kvtest.inputdir";
-  public static final String NATIVETASK_KVTEST_OUTPUTDIR = "nativetask.kvtest.outputdir";
-  public static final String NATIVETASK_KVTEST_NORMAL_OUTPUTDIR = "normal.kvtest.outputdir";
+  // kv test
+  public static final String FILESIZE_KEY = "kvtest.file.size";
+  public static final String NATIVETASK_KVTEST_DIR = NATIVETASK_TEST_DIR + "/kvtest";
+  public static final String NATIVETASK_KVTEST_INPUTDIR = NATIVETASK_KVTEST_DIR + "/input";
+  public static final String NATIVETASK_KVTEST_NATIVE_OUTPUTDIR = NATIVETASK_KVTEST_DIR + "/output/native";
+  public static final String NATIVETASK_KVTEST_NORMAL_OUTPUTDIR = NATIVETASK_KVTEST_DIR + "/output/normal";
   public static final String NATIVETASK_KVTEST_CREATEFILE = "nativetask.kvtest.createfile";
   public static final String NATIVETASK_KVTEST_FILE_RECORDNUM = "nativetask.kvtest.file.recordnum";
   public static final String NATIVETASK_KVTEST_KEYCLASSES = "nativetask.kvtest.keyclasses";
   public static final String NATIVETASK_KVTEST_VALUECLASSES = "nativetask.kvtest.valueclasses";
-  public static final String NATIVETASK_COLLECTOR_DELEGATOR = "mapreduce.map.output.collector.delegator.class";
-  public static final String NATIVETASK_COLLECTOR_DELEGATOR_CLASS = "org.apache.hadoop.mapred.nativetask.testutil.EnforceNativeOutputCollectorDelegator";
 
-  public static final String SNAPPY_COMPRESS_CONF_PATH = "test-snappy-compress-conf.xml";
-  public static final String GZIP_COMPRESS_CONF_PATH = "test-gzip-compress-conf.xml";
-  public static final String BZIP2_COMPRESS_CONF_PATH = "test-bzip2-compress-conf.xml";
-  public static final String DEFAULT_COMPRESS_CONF_PATH = "test-default-compress-conf.xml";
-  public static final String LZ4_COMPRESS_CONF_PATH = "test-lz4-compress-conf.xml";
+  // compress test
   public static final String NATIVETASK_COMPRESS_FILESIZE = "nativetask.compress.filesize";
+  public static final String NATIVETASK_COMPRESS_TEST_DIR = NATIVETASK_TEST_DIR + "/compresstest";
+  public static final String NATIVETASK_COMPRESS_TEST_INPUTDIR = NATIVETASK_COMPRESS_TEST_DIR + "/input";
+  public static final String NATIVETASK_COMPRESS_TEST_NATIVE_OUTPUTDIR = NATIVETASK_COMPRESS_TEST_DIR + "/output/native";
+  public static final String NATIVETASK_COMPRESS_TEST_NORMAL_OUTPUTDIR = NATIVETASK_COMPRESS_TEST_DIR + "/output/normal";
 
-  public static final String NATIVETASK_TEST_COMBINER_INPUTPATH_KEY = "nativetask.combinertest.inputpath";
-  public static final String NATIVETASK_TEST_COMBINER_INPUTPATH_DEFAULTV = "./combinertest/input";
-  public static final String NATIVETASK_TEST_COMBINER_OUTPUTPATH = "nativetask.combinertest.outputdir";
-  public static final String NATIVETASK_TEST_COMBINER_OUTPUTPATH_DEFAULTV = "./combinertest/output/native";
-  public static final String NORMAL_TEST_COMBINER_OUTPUTPATH = "normal.combinertest.outputdir";
-  public static final String NORMAL_TEST_COMBINER_OUTPUTPATH_DEFAULTV = "./combinertest/output/normal";
-  public static final String OLDAPI_NATIVETASK_TEST_COMBINER_OUTPUTPATH = "oldAPI.nativetask.combinertest.outputdir";
-  public static final String OLDAPI_NORMAL_TEST_COMBINER_OUTPUTPATH = "oldAPI.normal.combinertest.outputdir";
+  // combiner test
+  public static final String NATIVETASK_COMBINER_TEST_DIR = NATIVETASK_TEST_DIR + "/combinertest";
+  public static final String NATIVETASK_COMBINER_TEST_INPUTDIR = NATIVETASK_COMBINER_TEST_DIR + "/input";
+  public static final String NATIVETASK_COMBINER_TEST_NATIVE_OUTPUTDIR = NATIVETASK_COMBINER_TEST_DIR + "/output/native";
+  public static final String NATIVETASK_COMBINER_TEST_NORMAL_OUTPUTDIR = NATIVETASK_COMBINER_TEST_DIR + "/output/normal";
+  public static final String NATIVETASK_OLDAPI_COMBINER_TEST_NATIVE_OUTPUTPATH = NATIVETASK_COMBINER_TEST_DIR + "/oldapi/output/native";
+  public static final String NATIVETASK_OLDAPI_COMBINER_TEST_NORMAL_OUTPUTPATH = NATIVETASK_COMBINER_TEST_DIR + "/oldapi/output/normal";
   public static final String NATIVETASK_COMBINER_WORDCOUNT_FILESIZE = "nativetask.combiner.wordcount.filesize";
   public static final String NATIVETASK_NONSORTTEST_FILESIZE = "nativetask.nonsorttest.filesize";
 
-  public static final String COMMON_CONF_PATH = "common_conf.xml";
-
-  public static final String FILESIZE_KEY = "kvtest.file.size";
-  public static final String NATIVETASK_KVSIZE_MAX_LARGEKV_TEST = "nativetask.kvsize.max.largekv";
-
+  // nonsort test
   public static final String NATIVETASK_MAP_OUTPUT_SORT = "mapreduce.sort.avoidance";
-  public static final String NONSORT_TEST_INPUTDIR = "nativetask.nonsorttest.inputpath";
-  public static final String NONSORT_TEST_NATIVE_OUTPUT = "nonsorttest.native.outputdir";
-  public static final String NONSORT_TEST_NORMAL_OUTPUT = "nonsorttest.normal.outputdir";
+  public static final String NATIVETASK_NONSORT_TEST_DIR = NATIVETASK_TEST_DIR + "/nonsorttest";
+  public static final String NATIVETASK_NONSORT_TEST_INPUTDIR = NATIVETASK_NONSORT_TEST_DIR + "/input";
+  public static final String NATIVETASK_NONSORT_TEST_NATIVE_OUTPUT = NATIVETASK_NONSORT_TEST_DIR + "/output/native";
+  public static final String NATIVETASK_NONSORT_TEST_NORMAL_OUTPUT = NATIVETASK_NONSORT_TEST_DIR + "/output/normal";
 
 }

+ 40 - 31
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/common_conf.xml

@@ -1,39 +1,48 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
-<!-- Put site-specific property overrides in this file. -->
+<!--
+   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.
+-->
 
 <configuration>
-<property>
-<name>mapred.job.tracker</name>
-<value>local</value>
-</property>
-
-<property>
-<name>keep.failed.task.files</name>
-<value>true</value>
-</property>
-
   <property>
-  <name>keep.task.files.pattern</name>
-  <value>.*_m_</value>
-  <description>Keep all files from tasks whose task names match the given
-               regular expression. Defaults to none.</description>
+    <name>mapred.job.tracker</name>
+    <value>local</value>
+  </property>
+  <property>
+    <name>keep.failed.task.files</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>keep.task.files.pattern</name>
+    <value>.*_m_</value>
+    <description>Keep all files from tasks whose task names match the given
+      regular expression. Defaults to none.</description>
+  </property>
+  <property>
+    <name>nativetask.kvsize.max.largekv</name>
+    <value>1048576</value>
+  </property>
+  <property>
+    <name>native.processor.buffer.kb</name>
+    <value>128</value>
+  </property>
+  <property>
+    <name>nativetask.output.manager</name>
+    <value>org.apache.hadoop.mapred.nativetask.util.LocalJobOutputFiles</value>
   </property>
-
-<property>
-<name>nativetask.kvsize.max.largekv</name>
-<value>1048576</value>
-</property>
-
-<property>
-<name>native.processor.buffer.kb</name>
-<value>128</value>
-</property>
-
-<property>
-<name>nativetask.output.manager</name>
-<value>org.apache.hadoop.mapred.nativetask.util.LocalJobOutputFiles</value>
-</property>
-
 </configuration>

+ 78 - 80
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/kvtest-conf.xml

@@ -1,87 +1,85 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-<property>
-<name>nativetask.kvtest.inputdir</name>
-<value>./kvtest/input</value>
-</property>
-<property>
-<name>nativetask.kvtest.outputdir</name>
-<value>./kvtest/output/native</value>
-</property>
-<property>
-<name>normal.kvtest.outputdir</name>
-<value>./kvtest/output/normal</value>
-</property>
-<property>
-<name>kvtest.file.size</name>
-<value>10485760</value>
-</property>
-<property>
-<name>mapreduce.reduce.class</name>
-<value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
-</property>
-<property>
-<name>mapred.output.value.class</name>
-<value>org.apache.hadoop.io.IntWritable</value>
-</property>
-<property>
-<name>nativetask.kvtest.keyclasses</name>
-<value>
-org.apache.hadoop.io.BytesWritable;
-org.apache.hadoop.io.BooleanWritable;
-org.apache.hadoop.io.ByteWritable;
-org.apache.hadoop.io.DoubleWritable;
-org.apache.hadoop.io.FloatWritable;
-org.apache.hadoop.io.IntWritable;
-org.apache.hadoop.io.LongWritable;
-org.apache.hadoop.io.Text;
-org.apache.hadoop.io.VIntWritable;
-org.apache.hadoop.io.VLongWritable;
-</value>
-</property>
-<property>
-<name>nativetask.kvtest.valueclasses</name>
-<value>
-org.apache.hadoop.io.BytesWritable;
-org.apache.hadoop.io.BooleanWritable;
-org.apache.hadoop.io.ByteWritable;
-org.apache.hadoop.io.DoubleWritable;
-org.apache.hadoop.io.FloatWritable;
-org.apache.hadoop.io.IntWritable;
-org.apache.hadoop.io.LongWritable;
-org.apache.hadoop.io.Text;
-org.apache.hadoop.io.VIntWritable;
-org.apache.hadoop.io.VLongWritable;
-org.apache.hadoop.mapred.nativetask.testutil.MockValueClass;
-</value>
-</property>
-
-<property>
-<name>mapred.output.compress</name>
-<value>false</value>
-</property>
+<!--
+   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
 
-<property>
-<name>mapred.output.compression.type</name>
-<value>BLOCK</value>
-</property>
-<property>
-<name>mapred.compress.map.output</name>
-<value>true</value>
-</property>
-<property>
-<name>mapred.map.output.compression.codec</name>
-<value>org.apache.hadoop.io.compress.SnappyCodec</value>
-</property>
+   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.
+-->
 
-
-<property>
-<name>hadoop.native.lib</name>
-<value>true</value>
-</property>
+<configuration>
+  <property>
+    <name>kvtest.file.size</name>
+    <value>10485760</value>
+  </property>
+  <property>
+    <name>mapreduce.reduce.class</name>
+    <value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
+  </property>
+  <property>
+    <name>mapred.output.value.class</name>
+    <value>org.apache.hadoop.io.IntWritable</value>
+  </property>
+  <property>
+    <name>nativetask.kvtest.keyclasses</name>
+    <value>
+      org.apache.hadoop.io.BytesWritable;
+      org.apache.hadoop.io.BooleanWritable;
+      org.apache.hadoop.io.ByteWritable;
+      org.apache.hadoop.io.DoubleWritable;
+      org.apache.hadoop.io.FloatWritable;
+      org.apache.hadoop.io.IntWritable;
+      org.apache.hadoop.io.LongWritable;
+      org.apache.hadoop.io.Text;
+      org.apache.hadoop.io.VIntWritable;
+      org.apache.hadoop.io.VLongWritable;
+    </value>
+  </property>
+  <property>
+    <name>nativetask.kvtest.valueclasses</name>
+    <value>
+      org.apache.hadoop.io.BytesWritable;
+      org.apache.hadoop.io.BooleanWritable;
+      org.apache.hadoop.io.ByteWritable;
+      org.apache.hadoop.io.DoubleWritable;
+      org.apache.hadoop.io.FloatWritable;
+      org.apache.hadoop.io.IntWritable;
+      org.apache.hadoop.io.LongWritable;
+      org.apache.hadoop.io.Text;
+      org.apache.hadoop.io.VIntWritable;
+      org.apache.hadoop.io.VLongWritable;
+      org.apache.hadoop.mapred.nativetask.testutil.MockValueClass;
+    </value>
+  </property>
+  <property>
+    <name>mapred.output.compress</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>mapred.output.compression.type</name>
+    <value>BLOCK</value>
+  </property>
+  <property>
+    <name>mapred.compress.map.output</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>mapred.map.output.compression.codec</name>
+    <value>org.apache.hadoop.io.compress.SnappyCodec</value>
+  </property>
+  <property>
+    <name>hadoop.native.lib</name>
+    <value>true</value>
+  </property>
 </configuration>

+ 24 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/native_conf.xml

@@ -1,15 +1,30 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<configuration>
 
-<property>
-<name>mapreduce.job.map.output.collector.class</name>
-<value>org.apache.hadoop.mapred.nativetask.testutil.EnforceNativeOutputCollectorDelegator</value>
-</property>
+<!--
+   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
 
-<property>
-<name>io.sort.mb</name>
-<value>5</value>
-</property>
+   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.
+-->
 
+<configuration>
+  <property>
+    <name>mapreduce.job.map.output.collector.class</name>
+    <value>org.apache.hadoop.mapred.nativetask.testutil.EnforceNativeOutputCollectorDelegator</value>
+  </property>
+  <property>
+    <name>io.sort.mb</name>
+    <value>5</value>
+  </property>
 </configuration>

+ 21 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/normal_conf.xml

@@ -1,10 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<configuration>
 
-<property>
-<name>io.sort.mb</name>
-<value>30</value>
-</property>
+<!--
+   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.
+-->
+
+<configuration>
+  <property>
+    <name>io.sort.mb</name>
+    <value>30</value>
+  </property>
 </configuration>

+ 24 - 30
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-combiner-conf.xml

@@ -1,36 +1,30 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
-<!-- Put site-specific property overrides in this file. -->
+<!--
+   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
 
-<configuration>
-<property>
-<name>nativetask.combinertest.inputpath</name>
-<value>./combinertest/input</value>
-</property>
-<property>
-<name>nativetask.combiner.wordcount.filesize</name>
-<value>20971520</value>
-</property>
+       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.
+-->
 
-<property>
-<name>nativetask.combinertest.outputdir</name>
-<value>./combinertest/output/native</value>
-</property>
-<property>
-<name>normal.combinertest.outputdir</name>
-<value>./combinertest/output/normal</value>
-</property>
-<property>
-<name>oldAPI.nativetask.combinertest.outputdir</name>
-<value>./combinertest/oldapi/output/native</value>
-</property>
-<property>
-<name>oldAPI.normal.combinertest.outputdir</name>
-<value>./combinertest/oldapi/output/normal</value>
-</property>
-<property>
-<name>mapred.job.tracker</name>
-<value>local</value>
-</property>
+<configuration>
+  <property>
+    <name>nativetask.combiner.wordcount.filesize</name>
+    <value>20971520</value>
+  </property>
+  <property>
+    <name>mapred.job.tracker</name>
+    <value>local</value>
+  </property>
 </configuration>

+ 50 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-compress-conf.xml

@@ -0,0 +1,50 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+   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.
+-->
+
+<configuration>
+  <property>
+    <name>mapred.output.compress</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>mapred.output.value.class</name>
+    <value>org.apache.hadoop.io.IntWritable</value>
+  </property>
+  <property>
+    <name>nativetask.compress.filesize</name>
+    <value>100000</value>
+  </property>
+  <property>
+    <name>mapreduce.reduce.class</name>
+    <value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
+  </property>
+  <property>
+    <name>mapred.compress.map.output</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>mapred.output.compression.type</name>
+    <value>BLOCK</value>
+  </property>
+  <property>
+    <name>hadoop.native.lib</name>
+    <value>true</value>
+  </property>
+</configuration>

+ 0 - 39
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-gzip-compress-conf.xml

@@ -1,39 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-<property>
-<name>mapred.output.compress</name>
-<value>false</value>
-</property>
-<property>
-<name>mapred.output.value.class</name>
-<value>org.apache.hadoop.io.IntWritable</value>
-</property>
-<property>
-<name>nativetask.compress.filesize</name>
-<value>100000</value>
-</property>
-<property>
-<name>mapreduce.reduce.class</name>
-<value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
-</property>
-<property>
-<name>mapred.compress.map.output</name>
-<value>true</value>
-</property>
-<property>
-<name>mapred.output.compression.type</name>
-<value>BLOCK</value>
-</property>
-<property>
-<name>mapred.map.output.compression.codec</name>
-<value>org.apache.hadoop.io.compress.GzipCodec</value>
-</property>
-<property>
-<name>hadoop.native.lib</name>
-<value>true</value>
-</property>
-</configuration>

+ 0 - 39
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-lz4-compress-conf.xml

@@ -1,39 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-<property>
-<name>mapred.output.compress</name>
-<value>false</value>
-</property>
-<property>
-<name>mapred.output.value.class</name>
-<value>org.apache.hadoop.io.IntWritable</value>
-</property>
-<property>
-<name>nativetask.compress.filesize</name>
-<value>100000</value>
-</property>
-<property>
-<name>mapreduce.reduce.class</name>
-<value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
-</property>
-<property>
-<name>mapred.compress.map.output</name>
-<value>true</value>
-</property>
-<property>
-<name>mapred.output.compression.type</name>
-<value>BLOCK</value>
-</property>
-<property>
-<name>mapred.map.output.compression.codec</name>
-<value>org.apache.hadoop.io.compress.Lz4Codec</value>
-</property>
-<property>
-<name>hadoop.native.lib</name>
-<value>true</value>
-</property>
-</configuration>

+ 24 - 25
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-nonsort-conf.xml

@@ -1,31 +1,30 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
-<!-- Put site-specific property overrides in this file. -->
+<!--
+   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.
+-->
 
 <configuration>
-<property>
-<name>nativetask.nonsorttest.inputpath</name>
-<value>./sorttest/input</value>
-</property>
-<property>
-<name>nativetask.nonsorttest.filesize</name>
-<value>4194304</value>
-</property>
-<property>
-<name>nonsorttest.native.outputdir.tmp</name>
-<value>./sorttest/output/tmp</value>
-</property>
-<property>
-<name>nonsorttest.native.outputdir</name>
-<value>./sorttest/output/native</value>
-</property>
-<property>
-<name>nonsorttest.normal.outputdir</name>
-<value>./sort/output/normal</value>
-</property>
-<property>
-<name>mapred.job.tracker</name>
-<value>local</value>
-</property>
+  <property>
+    <name>nativetask.nonsorttest.filesize</name>
+    <value>4194304</value>
+  </property>
+  <property>
+    <name>mapred.job.tracker</name>
+    <value>local</value>
+  </property>
 </configuration>

+ 0 - 41
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/resources/test-snappy-compress-conf.xml

@@ -1,41 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-<property>
-<name>mapred.output.compress</name>
-<value>false</value>
-</property>
-<property>
-<name>mapreduce.reduce.class</name>
-<value>org.apache.hadoop.mapred.nativetask.kvtest.HashSumReducer</value>
-</property>
-
-<property>
-<name>mapred.output.value.class</name>
-<value>org.apache.hadoop.io.IntWritable</value>
-</property>
-<property>
-<name>nativetask.compress.filesize</name>
-<value>100000</value>
-</property>
-
-<property>
-<name>mapred.compress.map.output</name>
-<value>true</value>
-</property>
-<property>
-<name>mapred.output.compression.type</name>
-<value>BLOCK</value>
-</property>
-<property>
-<name>mapred.map.output.compression.codec</name>
-<value>org.apache.hadoop.io.compress.SnappyCodec</value>
-</property>
-<property>
-<name>hadoop.native.lib</name>
-<value>true</value>
-</property>
-</configuration>