Przeglądaj źródła

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 lat temu
rodzic
commit
7ade9b04ea
21 zmienionych plików z 588 dodań i 580 usunięć
  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>