HBASE-2495 Allow record filtering with selected row key values in HBase Export

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1081889 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-03-15 18:25:32 +00:00
parent b56b88e984
commit b408cc1217
2 changed files with 73 additions and 38 deletions

View File

@ -109,6 +109,8 @@ Release 0.91.0 - Unreleased
HBASE-3618 Add to HBase book, 'schema' chapter - pre-creating regions and HBASE-3618 Add to HBase book, 'schema' chapter - pre-creating regions and
key types (Doug Meil via Stack) key types (Doug Meil via Stack)
HBASE-3610 Improve RegionSplitter performance HBASE-3610 Improve RegionSplitter performance
HBASE-2495 Allow record filtering with selected row key values in HBase
Export (Subbu M Iyer via Stack)
TASK TASK
HBASE-3559 Move report of split to master OFF the heartbeat channel HBASE-3559 Move report of split to master OFF the heartbeat channel

View File

@ -26,6 +26,11 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
@ -84,20 +89,8 @@ public class Export {
Job job = new Job(conf, NAME + "_" + tableName); Job job = new Job(conf, NAME + "_" + tableName);
job.setJobName(NAME + "_" + tableName); job.setJobName(NAME + "_" + tableName);
job.setJarByClass(Exporter.class); job.setJarByClass(Exporter.class);
// TODO: Allow passing filter and subset of rows/columns. // Set optional scan parameters
Scan s = new Scan(); Scan s = getConfiguredScanForJob(conf, args);
// Optional arguments.
int versions = args.length > 2? Integer.parseInt(args[2]): 1;
s.setMaxVersions(versions);
long startTime = args.length > 3? Long.parseLong(args[3]): 0L;
long endTime = args.length > 4? Long.parseLong(args[4]): Long.MAX_VALUE;
s.setTimeRange(startTime, endTime);
s.setCacheBlocks(false);
if (conf.get(TableInputFormat.SCAN_COLUMN_FAMILY) != null) {
s.addFamily(Bytes.toBytes(conf.get(TableInputFormat.SCAN_COLUMN_FAMILY)));
}
LOG.info("verisons=" + versions + ", starttime=" + startTime +
", endtime=" + endTime);
TableMapReduceUtil.initTableMapperJob(tableName, s, Exporter.class, null, TableMapReduceUtil.initTableMapperJob(tableName, s, Exporter.class, null,
null, job); null, job);
// No reducers. Just write straight to output files. // No reducers. Just write straight to output files.
@ -109,6 +102,46 @@ public class Export {
return job; return job;
} }
private static Scan getConfiguredScanForJob(Configuration conf, String[] args) throws IOException {
Scan s = new Scan();
// Optional arguments.
// Set Scan Versions
int versions = args.length > 2? Integer.parseInt(args[2]): 1;
s.setMaxVersions(versions);
// Set Scan Range
long startTime = args.length > 3? Long.parseLong(args[3]): 0L;
long endTime = args.length > 4? Long.parseLong(args[4]): Long.MAX_VALUE;
s.setTimeRange(startTime, endTime);
// Set cache blocks
s.setCacheBlocks(false);
// Set Scan Column Family
if (conf.get(TableInputFormat.SCAN_COLUMN_FAMILY) != null) {
s.addFamily(Bytes.toBytes(conf.get(TableInputFormat.SCAN_COLUMN_FAMILY)));
}
// Set RowFilter or Prefix Filter if applicable.
Filter exportFilter = getExportFilter(args);
if (exportFilter!= null) {
LOG.info("Setting Scan Filter for Export.");
s.setFilter(exportFilter);
}
LOG.info("verisons=" + versions + ", starttime=" + startTime +
", endtime=" + endTime);
return s;
}
private static Filter getExportFilter(String[] args) {
Filter exportFilter = null;
String filterCriteria = (args.length > 5) ? args[5]: null;
if (filterCriteria == null) return null;
if (filterCriteria.startsWith("^")) {
String regexPattern = filterCriteria.substring(1, filterCriteria.length());
exportFilter = new RowFilter(CompareOp.EQUAL, new RegexStringComparator(regexPattern));
} else {
exportFilter = new PrefixFilter(Bytes.toBytes(filterCriteria));
}
return exportFilter;
}
/* /*
* @param errorMsg Error message. Can be null. * @param errorMsg Error message. Can be null.
*/ */
@ -117,7 +150,7 @@ public class Export {
System.err.println("ERROR: " + errorMsg); System.err.println("ERROR: " + errorMsg);
} }
System.err.println("Usage: Export [-D <property=value>]* <tablename> <outputdir> [<versions> " + System.err.println("Usage: Export [-D <property=value>]* <tablename> <outputdir> [<versions> " +
"[<starttime> [<endtime>]]]\n"); "[<starttime> [<endtime>]] [^[regex pattern] or [Prefix] to filter]]\n");
System.err.println(" Note: -D properties will be applied to the conf used. "); System.err.println(" Note: -D properties will be applied to the conf used. ");
System.err.println(" For example: "); System.err.println(" For example: ");
System.err.println(" -D mapred.output.compress=true"); System.err.println(" -D mapred.output.compress=true");