Browse Source

MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is sucessful (Contributed by Binglin Chang)

Binglin Chang 10 years ago
parent
commit
17cd0faace
10 changed files with 74 additions and 43 deletions
  1. 1 0
      hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt
  2. 2 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
  3. 8 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/CombinerTest.java
  4. 13 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
  5. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/OldAPICombinerTest.java
  6. 12 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/compresstest/CompressTest.java
  7. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
  8. 13 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
  9. 13 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
  10. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/nonsorttest/NonSortTest.java

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

@@ -17,3 +17,4 @@ MAPREDUCE-6026. native-task: fix logging (Manu Zhang via todd)
 MAPREDUCE-6035. native-task: sources/test-sources jar distribution (Manu Zhang via todd)
 MAPREDUCE-6035. native-task: sources/test-sources jar distribution (Manu Zhang via todd)
 MAPREDUCE-5977. Fix or suppress native-task gcc warnings (Manu Zhang via todd)
 MAPREDUCE-5977. Fix or suppress native-task gcc warnings (Manu Zhang via todd)
 MAPREDUCE-6054. native-task: Speed up tests (todd)
 MAPREDUCE-6054. native-task: Speed up tests (todd)
+MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is sucessful (Binglin Chang)

+ 2 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml

@@ -83,10 +83,8 @@
             </property>
             </property>
           </properties>
           </properties>
           <includes>
           <includes>
-            <include>**/TestTaskContext.java</include>
-            <include>**/buffer/Test*.java</include>
-            <include>**/handlers/Test*.java</include>
-            <include>**/serde/Test*.java</include>
+            <include>**/*Test.java</include>
+            <include>**/Test*.java</include>
           </includes>
           </includes>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
@@ -201,28 +199,6 @@
               </execution>
               </execution>
             </executions>
             </executions>
           </plugin>
           </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <configuration>
-              <properties>
-                <property>
-                  <name>listener</name>
-                  <value>org.apache.hadoop.test.TimedOutTestsListener</value>
-                </property>
-              </properties>
-              <includes>
-                <include>**/TestTaskContext.java</include>
-                <include>**/buffer/Test*.java</include>
-                <include>**/handlers/Test*.java</include>
-                <include>**/serde/Test*.java</include>
-                <include>**/combinertest/*Test.java</include>
-                <include>**/compresstest/*Test.java</include>
-                <include>**/nonsorttest/*Test.java</include>
-                <include>**/kvtest/*Test.java</include>
-              </includes>
-            </configuration>
-          </plugin>
         </plugins>
         </plugins>
       </build>
       </build>
     </profile>
     </profile>

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.mapred.nativetask.combinertest;
 package org.apache.hadoop.mapred.nativetask.combinertest;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -25,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapred.Task;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.combinertest.WordCount.IntSumReducer;
 import org.apache.hadoop.mapred.nativetask.combinertest.WordCount.IntSumReducer;
 import org.apache.hadoop.mapred.nativetask.combinertest.WordCount.TokenizerMapper;
 import org.apache.hadoop.mapred.nativetask.combinertest.WordCount.TokenizerMapper;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
@@ -36,6 +38,8 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -58,11 +62,11 @@ public class CombinerTest {
 
 
       final Job normaljob = getJob("normalwordcount", commonConf, inputpath, hadoopoutputpath);
       final Job normaljob = getJob("normalwordcount", commonConf, inputpath, hadoopoutputpath);
 
 
-      nativejob.waitForCompletion(true);
+      assertTrue(nativejob.waitForCompletion(true));
             
             
       Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
       Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
       
       
-      normaljob.waitForCompletion(true);
+      assertTrue(normaljob.waitForCompletion(true));
       Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
       Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
        
        
       assertEquals(true, ResultVerifier.verify(nativeoutputpath, hadoopoutputpath));
       assertEquals(true, ResultVerifier.verify(nativeoutputpath, hadoopoutputpath));
@@ -77,6 +81,8 @@ public class CombinerTest {
 
 
   @Before
   @Before
   public void startUp() throws Exception {
   public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     conf.addcombinerConf();
     conf.addcombinerConf();
 
 

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.mapred.nativetask.combinertest;
 package org.apache.hadoop.mapred.nativetask.combinertest;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -27,17 +28,27 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapred.Task;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class LargeKVCombinerTest {
 public class LargeKVCombinerTest {
   private static final Log LOG = LogFactory.getLog(LargeKVCombinerTest.class);
   private static final Log LOG = LogFactory.getLog(LargeKVCombinerTest.class);
 
 
+  @Before
+  public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
+  }
+
   @Test
   @Test
   public void testLargeValueCombiner(){
   public void testLargeValueCombiner(){
     final Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
     final Configuration normalConf = ScenarioConfiguration.getNormalConfiguration();
@@ -74,10 +85,10 @@ public class LargeKVCombinerTest {
         final Job normaljob = CombinerTest.getJob("normalwordcount", normalConf, inputPath, hadoopOutputPath);
         final Job normaljob = CombinerTest.getJob("normalwordcount", normalConf, inputPath, hadoopOutputPath);
         final Job nativejob = CombinerTest.getJob("nativewordcount", nativeConf, inputPath, nativeOutputPath);
         final Job nativejob = CombinerTest.getJob("nativewordcount", nativeConf, inputPath, nativeOutputPath);
         
         
-        nativejob.waitForCompletion(true);
+        assertTrue(nativejob.waitForCompletion(true));
         Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
         Counter nativeReduceGroups = nativejob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
         
         
-        normaljob.waitForCompletion(true);
+        assertTrue(normaljob.waitForCompletion(true));
         Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
         Counter normalReduceGroups = normaljob.getCounters().findCounter(Task.Counter.REDUCE_INPUT_RECORDS);
         
         
         final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);
         final boolean compareRet = ResultVerifier.verify(nativeOutputPath, hadoopOutputPath);

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

@@ -32,11 +32,14 @@ import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -70,6 +73,8 @@ public class OldAPICombinerTest {
 
 
   @Before
   @Before
   public void startUp() throws Exception {
   public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     conf.addcombinerConf();
     conf.addcombinerConf();
     this.fs = FileSystem.get(conf);
     this.fs = FileSystem.get(conf);

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

@@ -18,16 +18,20 @@
 package org.apache.hadoop.mapred.nativetask.compresstest;
 package org.apache.hadoop.mapred.nativetask.compresstest;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -38,12 +42,12 @@ public class CompressTest {
     final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
     final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
     conf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
     conf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
     final Job job = CompressMapper.getCompressJob("nativesnappy", conf);
     final Job job = CompressMapper.getCompressJob("nativesnappy", conf);
-    job.waitForCompletion(true);
+    assertTrue(job.waitForCompletion(true));
 
 
     final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
     final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
     hadoopconf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
     hadoopconf.addResource(TestConstants.SNAPPY_COMPRESS_CONF_PATH);
     final Job hadoopjob = CompressMapper.getCompressJob("hadoopsnappy", hadoopconf);
     final Job hadoopjob = CompressMapper.getCompressJob("hadoopsnappy", hadoopconf);
-    hadoopjob.waitForCompletion(true);
+    assertTrue(hadoopjob.waitForCompletion(true));
 
 
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativesnappy",
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativesnappy",
         CompressMapper.outputFileDir + "hadoopsnappy");
         CompressMapper.outputFileDir + "hadoopsnappy");
@@ -55,12 +59,12 @@ public class CompressTest {
     final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
     final Configuration conf = ScenarioConfiguration.getNativeConfiguration();
     conf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
     conf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
     final Job job = CompressMapper.getCompressJob("nativegzip", conf);
     final Job job = CompressMapper.getCompressJob("nativegzip", conf);
-    job.waitForCompletion(true);
+    assertTrue(job.waitForCompletion(true));
 
 
     final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
     final Configuration hadoopconf = ScenarioConfiguration.getNormalConfiguration();
     hadoopconf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
     hadoopconf.addResource(TestConstants.GZIP_COMPRESS_CONF_PATH);
     final Job hadoopjob = CompressMapper.getCompressJob("hadoopgzip", hadoopconf);
     final Job hadoopjob = CompressMapper.getCompressJob("hadoopgzip", hadoopconf);
-    hadoopjob.waitForCompletion(true);
+    assertTrue(hadoopjob.waitForCompletion(true));
 
 
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativegzip",
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativegzip",
         CompressMapper.outputFileDir + "hadoopgzip");
         CompressMapper.outputFileDir + "hadoopgzip");
@@ -72,12 +76,12 @@ public class CompressTest {
     final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
     final Configuration nativeConf = ScenarioConfiguration.getNativeConfiguration();
     nativeConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
     nativeConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
     final Job nativeJob = CompressMapper.getCompressJob("nativelz4", nativeConf);
     final Job nativeJob = CompressMapper.getCompressJob("nativelz4", nativeConf);
-    nativeJob.waitForCompletion(true);
+    assertTrue(nativeJob.waitForCompletion(true));
 
 
     final Configuration hadoopConf = ScenarioConfiguration.getNormalConfiguration();
     final Configuration hadoopConf = ScenarioConfiguration.getNormalConfiguration();
     hadoopConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
     hadoopConf.addResource(TestConstants.LZ4_COMPRESS_CONF_PATH);
     final Job hadoopJob = CompressMapper.getCompressJob("hadooplz4", hadoopConf);
     final Job hadoopJob = CompressMapper.getCompressJob("hadooplz4", hadoopConf);
-    hadoopJob.waitForCompletion(true);
+    assertTrue(hadoopJob.waitForCompletion(true));
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativelz4",
     final boolean compareRet = ResultVerifier.verify(CompressMapper.outputFileDir + "nativelz4",
         CompressMapper.outputFileDir + "hadooplz4");
         CompressMapper.outputFileDir + "hadooplz4");
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
     assertEquals("file compare result: if they are the same ,then return true", true, compareRet);
@@ -85,6 +89,8 @@ public class CompressTest {
 
 
   @Before
   @Before
   public void startUp() throws Exception {
   public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     final ScenarioConfiguration conf = new ScenarioConfiguration();
     final FileSystem fs = FileSystem.get(conf);
     final FileSystem fs = FileSystem.get(conf);
     final Path path = new Path(CompressMapper.inputFile);
     final Path path = new Path(CompressMapper.inputFile);

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java

@@ -96,8 +96,7 @@ public class KVJob {
     FileOutputFormat.setOutputPath(job, new Path(outputpath));
     FileOutputFormat.setOutputPath(job, new Path(outputpath));
   }
   }
 
 
-  public void runJob() throws Exception {
-
-    job.waitForCompletion(true);
+  public boolean runJob() throws Exception {
+    return job.waitForCompletion(true);
   }
   }
 }
 }

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.mapred.nativetask.kvtest;
 package org.apache.hadoop.mapred.nativetask.kvtest;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.List;
 import java.util.List;
@@ -28,9 +29,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized;
@@ -100,6 +105,12 @@ public class KVTest {
     this.valueclass = valueclass;
     this.valueclass = valueclass;
   }
   }
 
 
+  @Before
+  public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
+  }
+
   @Test
   @Test
   public void testKVCompability() {
   public void testKVCompability() {
     try {
     try {
@@ -139,7 +150,7 @@ public class KVTest {
     nativekvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "true");
     nativekvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "true");
     try {
     try {
       final KVJob keyJob = new KVJob(jobname, nativekvtestconf, keyclass, valueclass, inputpath, outputpath);
       final KVJob keyJob = new KVJob(jobname, nativekvtestconf, keyclass, valueclass, inputpath, outputpath);
-      keyJob.runJob();
+      assertTrue("job should complete successfully", keyJob.runJob());
     } catch (final Exception e) {
     } catch (final Exception e) {
       return "native testcase run time error.";
       return "native testcase run time error.";
     }
     }
@@ -161,7 +172,7 @@ public class KVTest {
     hadoopkvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "false");
     hadoopkvtestconf.set(TestConstants.NATIVETASK_KVTEST_CREATEFILE, "false");
     try {
     try {
       final KVJob keyJob = new KVJob(jobname, hadoopkvtestconf, keyclass, valueclass, inputpath, outputpath);
       final KVJob keyJob = new KVJob(jobname, hadoopkvtestconf, keyclass, valueclass, inputpath, outputpath);
-      keyJob.runJob();
+      assertTrue("job should complete successfully", keyJob.runJob());
     } catch (final Exception e) {
     } catch (final Exception e) {
       return "normal testcase run time error.";
       return "normal testcase run time error.";
     }
     }

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.mapred.nativetask.kvtest;
 package org.apache.hadoop.mapred.nativetask.kvtest;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -28,14 +29,24 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class LargeKVTest {
 public class LargeKVTest {
   private static final Log LOG = LogFactory.getLog(LargeKVTest.class);
   private static final Log LOG = LogFactory.getLog(LargeKVTest.class);
 
 
+  @Before
+  public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
+  }
+
   @Test
   @Test
   public void testKeySize() {
   public void testKeySize() {
     runKVSizeTests(Text.class, IntWritable.class);
     runKVSizeTests(Text.class, IntWritable.class);
@@ -104,7 +115,7 @@ public class LargeKVTest {
     fs.close();
     fs.close();
     try {
     try {
       final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
       final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
-      keyJob.runJob();
+      assertTrue("job should complete successfully", keyJob.runJob());
     } catch (final Exception e) {
     } catch (final Exception e) {
       return "normal testcase run time error.";
       return "normal testcase run time error.";
     }
     }
@@ -123,7 +134,7 @@ public class LargeKVTest {
     fs.close();
     fs.close();
     try {
     try {
       final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
       final KVJob keyJob = new KVJob(jobname, conf, keyclass, valueclass, inputpath, outputpath);
-      keyJob.runJob();
+      assertTrue("job should complete successfully", keyJob.runJob());
     } catch (final Exception e) {
     } catch (final Exception e) {
       return "normal testcase run time error.";
       return "normal testcase run time error.";
     }
     }

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.nativetask.NativeRuntime;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.kvtest.TestInputFile;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ResultVerifier;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
@@ -36,6 +37,8 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -65,6 +68,8 @@ public class NonSortTest {
 
 
   @Before
   @Before
   public void startUp() throws Exception {
   public void startUp() throws Exception {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Assume.assumeTrue(NativeRuntime.isNativeLibraryLoaded());
     final ScenarioConfiguration configuration = new ScenarioConfiguration();
     final ScenarioConfiguration configuration = new ScenarioConfiguration();
     configuration.addNonSortTestConf();
     configuration.addNonSortTestConf();
     final FileSystem fs = FileSystem.get(configuration);
     final FileSystem fs = FileSystem.get(configuration);