@InterfaceAudience.Public public class TableSnapshotInputFormat extends org.apache.hadoop.mapreduce.InputFormat<ImmutableBytesWritable,Result>
ExportSnapshot
tool, to a pure-hdfs cluster, and this
InputFormat can be used to run the mapreduce job directly over the snapshot files. The snapshot
should not be deleted while there are jobs reading from snapshot files.
Usage is similar to TableInputFormat, and
TableMapReduceUtil.initTableSnapshotMapperJob(String, Scan, Class, Class, Class, Job, boolean, Path)
can be used to configure the job.
{ @code Job job = new Job(conf); Scan scan = new Scan(); TableMapReduceUtil.initTableSnapshotMapperJob(snapshotName, scan, MyTableMapper.class, MyMapKeyOutput.class, MyMapOutputValueWritable.class, job, true); }
Internally, this input format restores the snapshot into the given tmp directory. By default, and
similar to TableInputFormat
an InputSplit is created per region, but optionally you can
run N mapper tasks per every region, in which case the region key range will be split to N
sub-ranges and an InputSplit will be created per sub-range. The region is opened for reading from
each RecordReader. An internal RegionScanner is used to execute the
CellScanner
obtained from the user.
HBase owns all the data and snapshot files on the filesystem. Only the 'hbase' user can read from snapshot files and data files. To read from snapshot files directly from the file system, the user who is running the MR job must have sufficient permissions to access snapshot and reference files. This means that to run mapreduce over snapshot files, the MR job has to be run as the HBase user or the user must have group or other privileges in the filesystem (See HBASE-8369). Note that, given other users access to read from snapshot/data files will completely circumvent the access control enforced by HBase.
TableSnapshotScanner
Modifier and Type | Class and Description |
---|---|
(package private) static class |
TableSnapshotInputFormat.TableSnapshotRegionRecordReader |
static class |
TableSnapshotInputFormat.TableSnapshotRegionSplit |
Constructor and Description |
---|
TableSnapshotInputFormat() |
Modifier and Type | Method and Description |
---|---|
static void |
cleanRestoreDir(org.apache.hadoop.mapreduce.Job job,
String snapshotName)
clean restore directory after snapshot scan job
|
org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable,Result> |
createRecordReader(org.apache.hadoop.mapreduce.InputSplit split,
org.apache.hadoop.mapreduce.TaskAttemptContext context) |
List<org.apache.hadoop.mapreduce.InputSplit> |
getSplits(org.apache.hadoop.mapreduce.JobContext job) |
static void |
setInput(org.apache.hadoop.mapreduce.Job job,
String snapshotName,
org.apache.hadoop.fs.Path restoreDir)
Configures the job to use TableSnapshotInputFormat to read from a snapshot.
|
static void |
setInput(org.apache.hadoop.mapreduce.Job job,
String snapshotName,
org.apache.hadoop.fs.Path restoreDir,
RegionSplitter.SplitAlgorithm splitAlgo,
int numSplitsPerRegion)
Configures the job to use TableSnapshotInputFormat to read from a snapshot.
|
public TableSnapshotInputFormat()
public org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable,Result> createRecordReader(org.apache.hadoop.mapreduce.InputSplit split, org.apache.hadoop.mapreduce.TaskAttemptContext context) throws IOException
createRecordReader
in class org.apache.hadoop.mapreduce.InputFormat<ImmutableBytesWritable,Result>
IOException
public List<org.apache.hadoop.mapreduce.InputSplit> getSplits(org.apache.hadoop.mapreduce.JobContext job) throws IOException, InterruptedException
getSplits
in class org.apache.hadoop.mapreduce.InputFormat<ImmutableBytesWritable,Result>
IOException
InterruptedException
public static void setInput(org.apache.hadoop.mapreduce.Job job, String snapshotName, org.apache.hadoop.fs.Path restoreDir) throws IOException
job
- the job to configuresnapshotName
- the name of the snapshot to read fromrestoreDir
- a temporary directory to restore the snapshot into. Current user should
have write permissions to this directory, and this should not be a
subdirectory of rootdir. After the job is finished, restoreDir can be
deleted.IOException
- if an error occurspublic static void setInput(org.apache.hadoop.mapreduce.Job job, String snapshotName, org.apache.hadoop.fs.Path restoreDir, RegionSplitter.SplitAlgorithm splitAlgo, int numSplitsPerRegion) throws IOException
job
- the job to configuresnapshotName
- the name of the snapshot to read fromrestoreDir
- a temporary directory to restore the snapshot into. Current user
should have write permissions to this directory, and this should not
be a subdirectory of rootdir. After the job is finished, restoreDir
can be deleted.splitAlgo
- split algorithm to generate splits from regionnumSplitsPerRegion
- how many input splits to generate per one regionIOException
- if an error occurspublic static void cleanRestoreDir(org.apache.hadoop.mapreduce.Job job, String snapshotName) throws IOException
job
- the snapshot scan jobsnapshotName
- the name of the snapshot to read fromIOException
- if an error occursCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.