HBASE-8094 TestTableInputFormatScan doesn't assert anything (Nick Dimiduk)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1456285 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Enis Soztutar 2013-03-14 00:03:41 +00:00
parent 88c3d9da6d
commit 566989a455
1 changed files with 5 additions and 11 deletions

View File

@ -18,7 +18,9 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import java.io.File;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableMap;
@ -26,7 +28,6 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
@ -39,16 +40,11 @@ import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Tests various scan start and stop row scenarios. This is set in a scan and
* tested in a MapReduce job to see if that is handed over and done properly
@ -346,8 +342,7 @@ public class TestTableInputFormatScan {
job.setInputFormatClass(TableInputFormat.class);
job.setNumReduceTasks(1);
FileOutputFormat.setOutputPath(job, new Path(job.getJobName()));
job.waitForCompletion(true);
assertTrue(job.isComplete());
assertTrue(job.waitForCompletion(true));
}
/**
@ -382,8 +377,7 @@ public class TestTableInputFormatScan {
job.setNumReduceTasks(1); // one to get final "first" and "last" key
FileOutputFormat.setOutputPath(job, new Path(job.getJobName()));
LOG.info("Started " + job.getJobName());
job.waitForCompletion(true);
assertTrue(job.isComplete());
assertTrue(job.waitForCompletion(true));
LOG.info("After map/reduce completion - job " + jobName);
}