HBASE-8453 TestImportExport failing again due to configuration issues -- REVERT
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1494351 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f7cd8f75e1
commit
fd4bd0c990
|
@ -2044,10 +2044,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
||||||
return dfsCluster;
|
return dfsCluster;
|
||||||
}
|
}
|
||||||
|
|
||||||
public MiniMRCluster getMRCluster() {
|
|
||||||
return mrCluster;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setDFSCluster(MiniDFSCluster cluster) throws IOException {
|
public void setDFSCluster(MiniDFSCluster cluster) throws IOException {
|
||||||
if (dfsCluster != null && dfsCluster.isClusterUp()) {
|
if (dfsCluster != null && dfsCluster.isClusterUp()) {
|
||||||
throw new IOException("DFSCluster is already running! Shut it down first.");
|
throw new IOException("DFSCluster is already running! Shut it down first.");
|
||||||
|
|
|
@ -31,7 +31,6 @@ import java.util.List;
|
||||||
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.hbase.HBaseConfiguration;
|
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
@ -51,7 +50,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
|
import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
|
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
|
||||||
import org.apache.hadoop.mapred.JobConf;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
import org.apache.hadoop.mapreduce.Mapper.Context;
|
||||||
import org.apache.hadoop.util.GenericOptionsParser;
|
import org.apache.hadoop.util.GenericOptionsParser;
|
||||||
|
@ -119,9 +117,7 @@ public class TestImportExport {
|
||||||
GenericOptionsParser opts = new GenericOptionsParser(new Configuration(UTIL.getConfiguration()), args);
|
GenericOptionsParser opts = new GenericOptionsParser(new Configuration(UTIL.getConfiguration()), args);
|
||||||
Configuration conf = opts.getConfiguration();
|
Configuration conf = opts.getConfiguration();
|
||||||
args = opts.getRemainingArgs();
|
args = opts.getRemainingArgs();
|
||||||
JobConf jobConf = UTIL.getMRCluster().createJobConf();
|
Job job = Export.createSubmittableJob(conf, args);
|
||||||
HBaseConfiguration.merge(jobConf, conf);
|
|
||||||
Job job = Export.createSubmittableJob(jobConf, args);
|
|
||||||
job.waitForCompletion(false);
|
job.waitForCompletion(false);
|
||||||
return job.isSuccessful();
|
return job.isSuccessful();
|
||||||
}
|
}
|
||||||
|
@ -139,9 +135,7 @@ public class TestImportExport {
|
||||||
GenericOptionsParser opts = new GenericOptionsParser(new Configuration(UTIL.getConfiguration()), args);
|
GenericOptionsParser opts = new GenericOptionsParser(new Configuration(UTIL.getConfiguration()), args);
|
||||||
Configuration conf = opts.getConfiguration();
|
Configuration conf = opts.getConfiguration();
|
||||||
args = opts.getRemainingArgs();
|
args = opts.getRemainingArgs();
|
||||||
JobConf jobConf = UTIL.getMRCluster().createJobConf();
|
Job job = Import.createSubmittableJob(conf, args);
|
||||||
HBaseConfiguration.merge(jobConf, conf);
|
|
||||||
Job job = Import.createSubmittableJob(jobConf, args);
|
|
||||||
job.waitForCompletion(false);
|
job.waitForCompletion(false);
|
||||||
return job.isSuccessful();
|
return job.isSuccessful();
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue