1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapred;
20
21 import org.apache.hadoop.fs.Path;
22 import org.apache.hadoop.hbase.HRegionInfo;
23 import org.apache.hadoop.hbase.HTableDescriptor;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.classification.InterfaceStability;
26 import org.apache.hadoop.hbase.client.Result;
27 import org.apache.hadoop.hbase.client.Scan;
28 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
29 import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl;
30 import org.apache.hadoop.hbase.util.RegionSplitter;
31 import org.apache.hadoop.mapred.InputFormat;
32 import org.apache.hadoop.mapred.InputSplit;
33 import org.apache.hadoop.mapred.JobConf;
34 import org.apache.hadoop.mapred.RecordReader;
35 import org.apache.hadoop.mapred.Reporter;
36 import org.apache.hadoop.mapreduce.Job;
37
38 import java.io.DataInput;
39 import java.io.DataOutput;
40 import java.io.IOException;
41 import java.util.List;
42
43
44
45
46
47
48
49 @InterfaceAudience.Public
50 @InterfaceStability.Evolving
51 public class TableSnapshotInputFormat implements InputFormat<ImmutableBytesWritable, Result> {
52
53 public static class TableSnapshotRegionSplit implements InputSplit {
54 private TableSnapshotInputFormatImpl.InputSplit delegate;
55
56
57 public TableSnapshotRegionSplit() {
58 this.delegate = new TableSnapshotInputFormatImpl.InputSplit();
59 }
60
61 public TableSnapshotRegionSplit(TableSnapshotInputFormatImpl.InputSplit delegate) {
62 this.delegate = delegate;
63 }
64
65 public TableSnapshotRegionSplit(HTableDescriptor htd, HRegionInfo regionInfo,
66 List<String> locations, Scan scan, Path restoreDir) {
67 this.delegate =
68 new TableSnapshotInputFormatImpl.InputSplit(htd, regionInfo, locations, scan, restoreDir);
69 }
70
71 @Override
72 public long getLength() throws IOException {
73 return delegate.getLength();
74 }
75
76 @Override
77 public String[] getLocations() throws IOException {
78 return delegate.getLocations();
79 }
80
81 @Override
82 public void write(DataOutput out) throws IOException {
83 delegate.write(out);
84 }
85
86 @Override
87 public void readFields(DataInput in) throws IOException {
88 delegate.readFields(in);
89 }
90 }
91
92 static class TableSnapshotRecordReader
93 implements RecordReader<ImmutableBytesWritable, Result> {
94
95 private TableSnapshotInputFormatImpl.RecordReader delegate;
96
97 public TableSnapshotRecordReader(TableSnapshotRegionSplit split, JobConf job)
98 throws IOException {
99 delegate = new TableSnapshotInputFormatImpl.RecordReader();
100 delegate.initialize(split.delegate, job);
101 }
102
103 @Override
104 public boolean next(ImmutableBytesWritable key, Result value) throws IOException {
105 if (!delegate.nextKeyValue()) {
106 return false;
107 }
108 ImmutableBytesWritable currentKey = delegate.getCurrentKey();
109 key.set(currentKey.get(), currentKey.getOffset(), currentKey.getLength());
110 value.copyFrom(delegate.getCurrentValue());
111 return true;
112 }
113
114 @Override
115 public ImmutableBytesWritable createKey() {
116 return new ImmutableBytesWritable();
117 }
118
119 @Override
120 public Result createValue() {
121 return new Result();
122 }
123
124 @Override
125 public long getPos() throws IOException {
126 return delegate.getPos();
127 }
128
129 @Override
130 public void close() throws IOException {
131 delegate.close();
132 }
133
134 @Override
135 public float getProgress() throws IOException {
136 return delegate.getProgress();
137 }
138 }
139
140 @Override
141 public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
142 List<TableSnapshotInputFormatImpl.InputSplit> splits =
143 TableSnapshotInputFormatImpl.getSplits(job);
144 InputSplit[] results = new InputSplit[splits.size()];
145 for (int i = 0; i < splits.size(); i++) {
146 results[i] = new TableSnapshotRegionSplit(splits.get(i));
147 }
148 return results;
149 }
150
151 @Override
152 public RecordReader<ImmutableBytesWritable, Result>
153 getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
154 return new TableSnapshotRecordReader((TableSnapshotRegionSplit) split, job);
155 }
156
157
158
159
160
161
162
163
164
165
166 public static void setInput(JobConf job, String snapshotName, Path restoreDir)
167 throws IOException {
168 TableSnapshotInputFormatImpl.setInput(job, snapshotName, restoreDir);
169 }
170
171
172
173
174
175
176
177
178
179
180
181
182 public static void setInput(JobConf job, String snapshotName, Path restoreDir,
183 RegionSplitter.SplitAlgorithm splitAlgo, int numSplitsPerRegion) throws IOException {
184 TableSnapshotInputFormatImpl.setInput(job, snapshotName, restoreDir, splitAlgo, numSplitsPerRegion);
185 }
186 }