HBASE-581 Allow adding filters to TableInputFormat (At same time, ensure TIF is subclassable)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@651368 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
fb2db8cdaa
commit
cecc9e99ff
|
@ -26,7 +26,8 @@ Hbase Change Log
|
|||
IMPROVEMENTS
|
||||
HBASE-559 MR example job to count table rows
|
||||
HBASE-596 DemoClient.py (Ivan Begtin via Stack)
|
||||
|
||||
HBASE-581 Allow adding filters to TableInputFormat (At same time, ensure TIF
|
||||
is subclassable) (David Alves via Stack)
|
||||
|
||||
Release 0.1.1 - 04/11/2008
|
||||
|
||||
|
|
|
@ -20,174 +20,44 @@
|
|||
package org.apache.hadoop.hbase.mapred;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.hadoop.io.MapWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.hadoop.mapred.InputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.JobConfigurable;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.io.RowResult;
|
||||
import org.apache.hadoop.hbase.io.Cell;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Scanner;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.JobConfigurable;
|
||||
|
||||
/**
|
||||
* Convert HBase tabular data into a format that is consumable by Map/Reduce
|
||||
* Convert HBase tabular data into a format that is consumable by Map/Reduce.
|
||||
*/
|
||||
public class TableInputFormat
|
||||
implements InputFormat<Text, RowResult>, JobConfigurable {
|
||||
protected final Log LOG = LogFactory.getLog(TableInputFormat.class);
|
||||
public class TableInputFormat extends TableInputFormatBase implements
|
||||
JobConfigurable {
|
||||
private final Log LOG = LogFactory.getLog(TableInputFormat.class);
|
||||
|
||||
/**
|
||||
* space delimited list of columns
|
||||
* @see org.apache.hadoop.hbase.regionserver.HAbstractScanner for column name wildcards
|
||||
* space delimited list of columns
|
||||
*
|
||||
* @see org.apache.hadoop.hbase.regionserver.HAbstractScanner for column name
|
||||
* wildcards
|
||||
*/
|
||||
public static final String COLUMN_LIST = "hbase.mapred.tablecolumns";
|
||||
|
||||
private Text m_tableName;
|
||||
Text[] m_cols;
|
||||
HTable m_table;
|
||||
|
||||
/**
|
||||
* Iterate over an HBase table data,
|
||||
* return (Text, RowResult) pairs
|
||||
*/
|
||||
class TableRecordReader implements RecordReader<Text, RowResult> {
|
||||
private final Scanner m_scanner;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param startRow (inclusive)
|
||||
* @param endRow (exclusive)
|
||||
* @throws IOException
|
||||
*/
|
||||
public TableRecordReader(Text startRow, Text endRow) throws IOException {
|
||||
if (endRow != null && endRow.getLength() > 0) {
|
||||
this.m_scanner = m_table.getScanner(m_cols, startRow, endRow);
|
||||
} else {
|
||||
this.m_scanner = m_table.getScanner(m_cols, startRow);
|
||||
}
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void close() throws IOException {
|
||||
this.m_scanner.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Text
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.RecordReader#createKey()
|
||||
*/
|
||||
public Text createKey() {
|
||||
return new Text();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return RowResult
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.RecordReader#createValue()
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public RowResult createValue() {
|
||||
return new RowResult();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public long getPos() {
|
||||
// This should be the ordinal tuple in the range;
|
||||
// not clear how to calculate...
|
||||
return 0;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public float getProgress() {
|
||||
// Depends on the total number of tuples and getPos
|
||||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key HStoreKey as input key.
|
||||
* @param value MapWritable as input value
|
||||
*
|
||||
* Converts Scanner.next() to
|
||||
* Text, RowResult
|
||||
*
|
||||
* @return true if there was more data
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public boolean next(Text key, RowResult value) throws IOException {
|
||||
RowResult result = m_scanner.next();
|
||||
boolean hasMore = result != null;
|
||||
if (hasMore) {
|
||||
Writables.copyWritable(result.getRow(), key);
|
||||
Writables.copyWritable(result, value);
|
||||
}
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public RecordReader<Text, RowResult> getRecordReader(
|
||||
InputSplit split,
|
||||
@SuppressWarnings("unused") JobConf job,
|
||||
@SuppressWarnings("unused") Reporter reporter)
|
||||
throws IOException {
|
||||
TableSplit tSplit = (TableSplit)split;
|
||||
return new TableRecordReader(tSplit.getStartRow(), tSplit.getEndRow());
|
||||
}
|
||||
|
||||
/**
|
||||
* A split will be created for each HRegion of the input table
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop.mapred.JobConf, int)
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
Text[] startKeys = m_table.getStartKeys();
|
||||
if(startKeys == null || startKeys.length == 0) {
|
||||
throw new IOException("Expecting at least one region");
|
||||
}
|
||||
InputSplit[] splits = new InputSplit[startKeys.length];
|
||||
for(int i = 0; i < startKeys.length; i++) {
|
||||
splits[i] = new TableSplit(m_tableName, startKeys[i],
|
||||
((i + 1) < startKeys.length) ? startKeys[i + 1] : new Text());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("split: " + i + "->" + splits[i]);
|
||||
}
|
||||
}
|
||||
return splits;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void configure(JobConf job) {
|
||||
Path[] tableNames = job.getInputPaths();
|
||||
m_tableName = new Text(tableNames[0].getName());
|
||||
String colArg = job.get(COLUMN_LIST);
|
||||
String[] colNames = colArg.split(" ");
|
||||
m_cols = new Text[colNames.length];
|
||||
for(int i = 0; i < m_cols.length; i++) {
|
||||
Text[] m_cols = new Text[colNames.length];
|
||||
for (int i = 0; i < m_cols.length; i++) {
|
||||
m_cols[i] = new Text(colNames[i]);
|
||||
}
|
||||
setInputColums(m_cols);
|
||||
try {
|
||||
m_table = new HTable(new HBaseConfiguration(job), m_tableName);
|
||||
setHTable(new HTable(new HBaseConfiguration(job), new Text(tableNames[0]
|
||||
.getName())));
|
||||
} catch (Exception e) {
|
||||
LOG.error(e);
|
||||
}
|
||||
|
@ -197,14 +67,14 @@ implements InputFormat<Text, RowResult>, JobConfigurable {
|
|||
public void validateInput(JobConf job) throws IOException {
|
||||
// expecting exactly one path
|
||||
Path[] tableNames = job.getInputPaths();
|
||||
if(tableNames == null || tableNames.length > 1) {
|
||||
if (tableNames == null || tableNames.length > 1) {
|
||||
throw new IOException("expecting one table name");
|
||||
}
|
||||
|
||||
// expecting at least one column
|
||||
String colArg = job.get(COLUMN_LIST);
|
||||
if(colArg == null || colArg.length() == 0) {
|
||||
if (colArg == null || colArg.length() == 0) {
|
||||
throw new IOException("expecting at least one column");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,312 @@
|
|||
/*
|
||||
* $Id$
|
||||
*
|
||||
* Copyright Critical Software S.A., All Rights Reserved.
|
||||
* (www.criticalsoftware.com)
|
||||
*
|
||||
* This software is the proprietary information of Critical Software S.A.
|
||||
* Use is subject to license terms.
|
||||
*
|
||||
* Last changed on : $Date$
|
||||
* Last changed by : $Author$
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mapred;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Scanner;
|
||||
import org.apache.hadoop.hbase.filter.RowFilterInterface;
|
||||
import org.apache.hadoop.hbase.filter.RowFilterSet;
|
||||
import org.apache.hadoop.hbase.filter.StopRowFilter;
|
||||
import org.apache.hadoop.hbase.io.RowResult;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.InputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
|
||||
/**
|
||||
* A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a
|
||||
* {@link Text}[] of input columns and optionally a {@link RowFilterInterface}.
|
||||
* Subclasses may use other {@link TableRecordReader} implementations.
|
||||
* <p>
|
||||
* An example of a subclass:
|
||||
* <code>
|
||||
* class ExampleTIF extends TableInputFormatBase implements JobConfigurable {
|
||||
*
|
||||
* public void configure(JobConf job) {
|
||||
* HTable exampleTable = new HTable(new HBaseConfiguration(job),
|
||||
* new Text("exampleTable"));
|
||||
* // mandatory
|
||||
* setHTable(exampleTable);
|
||||
* Text[] inputColumns = new Text[] { new Text("columnA"),
|
||||
* new Text("columnB") };
|
||||
* // mandatory
|
||||
* setInputColums(inputColumns);
|
||||
* RowFilterInterface exampleFilter = new RegExpRowFilter("keyPrefix.*");
|
||||
* // optional
|
||||
* setRowFilter(exampleFilter);
|
||||
* }
|
||||
*
|
||||
* public void validateInput(JobConf job) throws IOException {
|
||||
* }
|
||||
* }
|
||||
* </code>
|
||||
*/
|
||||
public abstract class TableInputFormatBase
|
||||
implements InputFormat<Text, RowResult> {
|
||||
private final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
|
||||
private Text[] inputColumns;
|
||||
private HTable table;
|
||||
private TableRecordReader tableRecordReader;
|
||||
private RowFilterInterface rowFilter;
|
||||
|
||||
/**
|
||||
* Iterate over an HBase table data, return (Text, RowResult) pairs
|
||||
*/
|
||||
protected class TableRecordReader implements RecordReader<Text, RowResult> {
|
||||
|
||||
private Text startRow;
|
||||
private Text endRow;
|
||||
private RowFilterInterface trrRowFilter;
|
||||
private Scanner scanner;
|
||||
private HTable htable;
|
||||
private Text[] trrInputColumns;
|
||||
|
||||
/**
|
||||
* Build the scanner. Not done in constructor to allow for extension.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public void init() throws IOException {
|
||||
if ((endRow != null) && (endRow.getLength() > 0)) {
|
||||
if (trrRowFilter != null) {
|
||||
final Set<RowFilterInterface> rowFiltersSet =
|
||||
new HashSet<RowFilterInterface>();
|
||||
rowFiltersSet.add(new StopRowFilter(endRow));
|
||||
rowFiltersSet.add(trrRowFilter);
|
||||
this.scanner = this.htable.getScanner(trrInputColumns, startRow,
|
||||
new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
|
||||
rowFiltersSet));
|
||||
} else {
|
||||
this.scanner =
|
||||
this.htable.getScanner(trrInputColumns, startRow, endRow);
|
||||
}
|
||||
} else {
|
||||
this.scanner =
|
||||
this.htable.getScanner(trrInputColumns, startRow, trrRowFilter);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param htable the {@link HTable} to scan.
|
||||
*/
|
||||
public void setHTable(HTable htable) {
|
||||
this.htable = htable;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param inputColumns the columns to be placed in {@link RowResult}.
|
||||
*/
|
||||
public void setInputColumns(Text[] inputColumns) {
|
||||
this.trrInputColumns = inputColumns;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param startRow the first row in the split
|
||||
*/
|
||||
public void setStartRow(Text startRow) {
|
||||
this.startRow = startRow;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param endRow the last row in the split
|
||||
*/
|
||||
public void setEndRow(Text endRow) {
|
||||
this.endRow = endRow;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param rowFilter the {@link RowFilterInterface} to be used.
|
||||
*/
|
||||
public void setRowFilter(RowFilterInterface rowFilter) {
|
||||
this.trrRowFilter = rowFilter;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public void close() throws IOException {
|
||||
this.scanner.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Text
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.RecordReader#createKey()
|
||||
*/
|
||||
public Text createKey() {
|
||||
return new Text();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return RowResult
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.RecordReader#createValue()
|
||||
*/
|
||||
public RowResult createValue() {
|
||||
return new RowResult();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public long getPos() {
|
||||
// This should be the ordinal tuple in the range;
|
||||
// not clear how to calculate...
|
||||
return 0;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public float getProgress() {
|
||||
// Depends on the total number of tuples and getPos
|
||||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key HStoreKey as input key.
|
||||
* @param value MapWritable as input value
|
||||
*
|
||||
* Converts Scanner.next() to Text, RowResult
|
||||
*
|
||||
* @return true if there was more data
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public boolean next(Text key, RowResult value) throws IOException {
|
||||
RowResult result = this.scanner.next();
|
||||
boolean hasMore = result != null;
|
||||
if (hasMore) {
|
||||
Writables.copyWritable(result.getRow(), key);
|
||||
Writables.copyWritable(result, value);
|
||||
}
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a {@link TableRecordReader}. If no {@link TableRecordReader} was
|
||||
* provided uses the default.
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit,
|
||||
* JobConf, Reporter)
|
||||
*/
|
||||
public RecordReader<Text, RowResult> getRecordReader(InputSplit split,
|
||||
@SuppressWarnings("unused")
|
||||
JobConf job, @SuppressWarnings("unused")
|
||||
Reporter reporter)
|
||||
throws IOException {
|
||||
TableSplit tSplit = (TableSplit) split;
|
||||
TableRecordReader trr = this.tableRecordReader;
|
||||
// if no table record reader was provided use default
|
||||
if (trr == null) {
|
||||
trr = new TableRecordReader();
|
||||
}
|
||||
trr.setStartRow(tSplit.getStartRow());
|
||||
trr.setEndRow(tSplit.getEndRow());
|
||||
trr.setHTable(this.table);
|
||||
trr.setInputColumns(this.inputColumns);
|
||||
trr.setRowFilter(this.rowFilter);
|
||||
trr.init();
|
||||
return trr;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the splits that will serve as input for the map tasks.
|
||||
* <ul>
|
||||
* Splits are created in number equal to the smallest between numSplits and
|
||||
* the number of {@link HRegion}s in the table. If the number of splits is
|
||||
* smaller than the number of {@link HRegion}s then splits are spanned across
|
||||
* multiple {@link HRegion}s and are grouped the most evenly possible. In the
|
||||
* case splits are uneven the bigger splits are placed first in the
|
||||
* {@link InputSplit} array.
|
||||
*
|
||||
* @param job the map task {@link JobConf}
|
||||
* @param numSplits a hint to calculate the number of splits
|
||||
*
|
||||
* @return the input splits
|
||||
*
|
||||
* @see org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop.mapred.JobConf, int)
|
||||
*/
|
||||
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
Text[] startKeys = this.table.getStartKeys();
|
||||
if (startKeys == null || startKeys.length == 0) {
|
||||
throw new IOException("Expecting at least one region");
|
||||
}
|
||||
if (this.table == null) {
|
||||
throw new IOException("No table was provided");
|
||||
}
|
||||
if (this.inputColumns == null || this.inputColumns.length == 0) {
|
||||
throw new IOException("Expecting at least one column");
|
||||
}
|
||||
int realNumSplits = numSplits > startKeys.length ? startKeys.length
|
||||
: numSplits;
|
||||
InputSplit[] splits = new InputSplit[realNumSplits];
|
||||
int middle = startKeys.length / realNumSplits;
|
||||
int startPos = 0;
|
||||
for (int i = 0; i < realNumSplits; i++) {
|
||||
int lastPos = startPos + middle;
|
||||
lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
|
||||
splits[i] = new TableSplit(this.table.getTableName(),
|
||||
startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]
|
||||
: new Text());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("split: " + i + "->" + splits[i]);
|
||||
}
|
||||
startPos = lastPos;
|
||||
}
|
||||
return splits;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* @param inputColumns to be passed in {@link RowResult} to the map task.
|
||||
*/
|
||||
protected void setInputColums(Text[] inputColumns) {
|
||||
this.inputColumns = inputColumns;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows subclasses to set the {@link HTable}.
|
||||
*
|
||||
* @param table to get the data from
|
||||
*/
|
||||
protected void setHTable(HTable table) {
|
||||
this.table = table;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows subclasses to set the {@link TableRecordReader}.
|
||||
*
|
||||
* @param tableRecordReader
|
||||
* to provide other {@link TableRecordReader} implementations.
|
||||
*/
|
||||
protected void setTableRecordReader(TableRecordReader tableRecordReader) {
|
||||
this.tableRecordReader = tableRecordReader;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows subclasses to set the {@link RowFilterInterface} to be used.
|
||||
*
|
||||
* @param rowFilter
|
||||
*/
|
||||
protected void setRowFilter(RowFilterInterface rowFilter) {
|
||||
this.rowFilter = rowFilter;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue