HBASE-3076 Allow to disable automatic shipping of dependency jars for mapreduce jobs

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1024317 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-10-19 16:44:06 +00:00
parent d4d7ceef68
commit b825581bc4
3 changed files with 114 additions and 7 deletions

View File

@ -1014,6 +1014,8 @@ Release 0.21.0 - Unreleased
HBASE-3097 Merge in hbase-1200 doc on bloomfilters into hbase book HBASE-3097 Merge in hbase-1200 doc on bloomfilters into hbase book
HBASE-2700 Test of: Handle master failover for regions in transition HBASE-2700 Test of: Handle master failover for regions in transition
HBASE-3115 HBaseClient wastes 1 TCP packet per RPC HBASE-3115 HBaseClient wastes 1 TCP packet per RPC
HBASE-3076 Allow to disable automatic shipping of dependency jars
for mapreduce jobs (Bruno Dumon)
NEW FEATURES NEW FEATURES
HBASE-1961 HBase EC2 scripts HBASE-1961 HBase EC2 scripts

View File

@ -56,6 +56,26 @@ public class TableMapReduceUtil {
Class<? extends TableMap> mapper, Class<? extends TableMap> mapper,
Class<? extends WritableComparable> outputKeyClass, Class<? extends WritableComparable> outputKeyClass,
Class<? extends Writable> outputValueClass, JobConf job) { Class<? extends Writable> outputValueClass, JobConf job) {
initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job, true);
}
/**
* Use this before submitting a TableMap job. It will
* appropriately set up the JobConf.
*
* @param table The table name to read from.
* @param columns The columns to scan.
* @param mapper The mapper class to use.
* @param outputKeyClass The class of the output key.
* @param outputValueClass The class of the output value.
* @param job The current job configuration to adjust.
* @param addDependencyJars upload HBase jars and jars for any of the configured
* job classes via the distributed cache (tmpjars).
*/
public static void initTableMapJob(String table, String columns,
Class<? extends TableMap> mapper,
Class<? extends WritableComparable> outputKeyClass,
Class<? extends Writable> outputValueClass, JobConf job, boolean addDependencyJars) {
job.setInputFormat(TableInputFormat.class); job.setInputFormat(TableInputFormat.class);
job.setMapOutputValueClass(outputValueClass); job.setMapOutputValueClass(outputValueClass);
@ -63,10 +83,12 @@ public class TableMapReduceUtil {
job.setMapperClass(mapper); job.setMapperClass(mapper);
FileInputFormat.addInputPaths(job, table); FileInputFormat.addInputPaths(job, table);
job.set(TableInputFormat.COLUMN_LIST, columns); job.set(TableInputFormat.COLUMN_LIST, columns);
try { if (addDependencyJars) {
addDependencyJars(job); try {
} catch (IOException e) { addDependencyJars(job);
e.printStackTrace(); } catch (IOException e) {
e.printStackTrace();
}
} }
} }
@ -99,6 +121,25 @@ public class TableMapReduceUtil {
public static void initTableReduceJob(String table, public static void initTableReduceJob(String table,
Class<? extends TableReduce> reducer, JobConf job, Class partitioner) Class<? extends TableReduce> reducer, JobConf job, Class partitioner)
throws IOException { throws IOException {
initTableReduceJob(table, reducer, job, partitioner, true);
}
/**
* Use this before submitting a TableReduce job. It will
* appropriately set up the JobConf.
*
* @param table The output table.
* @param reducer The reducer class to use.
* @param job The current job configuration to adjust.
* @param partitioner Partitioner to use. Pass <code>null</code> to use
* default partitioner.
* @param addDependencyJars upload HBase jars and jars for any of the configured
* job classes via the distributed cache (tmpjars).
* @throws IOException When determining the region count fails.
*/
public static void initTableReduceJob(String table,
Class<? extends TableReduce> reducer, JobConf job, Class partitioner,
boolean addDependencyJars) throws IOException {
job.setOutputFormat(TableOutputFormat.class); job.setOutputFormat(TableOutputFormat.class);
job.setReducerClass(reducer); job.setReducerClass(reducer);
job.set(TableOutputFormat.OUTPUT_TABLE, table); job.set(TableOutputFormat.OUTPUT_TABLE, table);
@ -114,7 +155,9 @@ public class TableMapReduceUtil {
} else if (partitioner != null) { } else if (partitioner != null) {
job.setPartitionerClass(partitioner); job.setPartitionerClass(partitioner);
} }
addDependencyJars(job); if (addDependencyJars) {
addDependencyJars(job);
}
} }
/** /**

View File

@ -69,6 +69,31 @@ public class TableMapReduceUtil {
Class<? extends TableMapper> mapper, Class<? extends TableMapper> mapper,
Class<? extends WritableComparable> outputKeyClass, Class<? extends WritableComparable> outputKeyClass,
Class<? extends Writable> outputValueClass, Job job) Class<? extends Writable> outputValueClass, Job job)
throws IOException {
initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
job, true);
}
/**
* Use this before submitting a TableMap job. It will appropriately set up
* the job.
*
* @param table The table name to read from.
* @param scan The scan instance with the columns, time range etc.
* @param mapper The mapper class to use.
* @param outputKeyClass The class of the output key.
* @param outputValueClass The class of the output value.
* @param job The current job to adjust. Make sure the passed job is
* carrying all necessary HBase configuration.
* @param addDependencyJars upload HBase jars and jars for any of the configured
* job classes via the distributed cache (tmpjars).
* @throws IOException When setting up the details fails.
*/
public static void initTableMapperJob(String table, Scan scan,
Class<? extends TableMapper> mapper,
Class<? extends WritableComparable> outputKeyClass,
Class<? extends Writable> outputValueClass, Job job,
boolean addDependencyJars)
throws IOException { throws IOException {
job.setInputFormatClass(TableInputFormat.class); job.setInputFormatClass(TableInputFormat.class);
if (outputValueClass != null) job.setMapOutputValueClass(outputValueClass); if (outputValueClass != null) job.setMapOutputValueClass(outputValueClass);
@ -77,7 +102,9 @@ public class TableMapReduceUtil {
job.getConfiguration().set(TableInputFormat.INPUT_TABLE, table); job.getConfiguration().set(TableInputFormat.INPUT_TABLE, table);
job.getConfiguration().set(TableInputFormat.SCAN, job.getConfiguration().set(TableInputFormat.SCAN,
convertScanToString(scan)); convertScanToString(scan));
addDependencyJars(job); if (addDependencyJars) {
addDependencyJars(job);
}
} }
/** /**
@ -167,6 +194,38 @@ public class TableMapReduceUtil {
Class<? extends TableReducer> reducer, Job job, Class<? extends TableReducer> reducer, Job job,
Class partitioner, String quorumAddress, String serverClass, Class partitioner, String quorumAddress, String serverClass,
String serverImpl) throws IOException { String serverImpl) throws IOException {
initTableReducerJob(table, reducer, job, partitioner, quorumAddress,
serverClass, serverImpl, true);
}
/**
* Use this before submitting a TableReduce job. It will
* appropriately set up the JobConf.
*
* @param table The output table.
* @param reducer The reducer class to use.
* @param job The current job to adjust. Make sure the passed job is
* carrying all necessary HBase configuration.
* @param partitioner Partitioner to use. Pass <code>null</code> to use
* default partitioner.
* @param quorumAddress Distant cluster to write to; default is null for
* output to the cluster that is designated in <code>hbase-site.xml</code>.
* Set this String to the zookeeper ensemble of an alternate remote cluster
* when you would have the reduce write a cluster that is other than the
* default; e.g. copying tables between clusters, the source would be
* designated by <code>hbase-site.xml</code> and this param would have the
* ensemble address of the remote cluster. The format to pass is particular.
* Pass <code> &lt;hbase.zookeeper.quorum> ':' &lt;ZOOKEEPER_ZNODE_PARENT></code>.
* @param serverClass redefined hbase.regionserver.class
* @param serverImpl redefined hbase.regionserver.impl
* @param addDependencyJars upload HBase jars and jars for any of the configured
* job classes via the distributed cache (tmpjars).
* @throws IOException When determining the region count fails.
*/
public static void initTableReducerJob(String table,
Class<? extends TableReducer> reducer, Job job,
Class partitioner, String quorumAddress, String serverClass,
String serverImpl, boolean addDependencyJars) throws IOException {
Configuration conf = job.getConfiguration(); Configuration conf = job.getConfiguration();
job.setOutputFormatClass(TableOutputFormat.class); job.setOutputFormatClass(TableOutputFormat.class);
@ -198,7 +257,10 @@ public class TableMapReduceUtil {
} else if (partitioner != null) { } else if (partitioner != null) {
job.setPartitionerClass(partitioner); job.setPartitionerClass(partitioner);
} }
addDependencyJars(job);
if (addDependencyJars) {
addDependencyJars(job);
}
} }
/** /**