1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
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.client.metrics.ScanMetrics;
29 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
30 import org.apache.hadoop.hbase.util.RegionSplitter;
31 import org.apache.hadoop.io.Writable;
32 import org.apache.hadoop.mapreduce.InputFormat;
33 import org.apache.hadoop.mapreduce.InputSplit;
34 import org.apache.hadoop.mapreduce.Job;
35 import org.apache.hadoop.mapreduce.JobContext;
36 import org.apache.hadoop.mapreduce.RecordReader;
37 import org.apache.hadoop.mapreduce.TaskAttemptContext;
38
39 import java.io.DataInput;
40 import java.io.DataOutput;
41 import java.io.IOException;
42 import java.lang.reflect.Method;
43 import java.util.ArrayList;
44 import java.util.List;
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86 @InterfaceAudience.Public
87 @InterfaceStability.Evolving
88 public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable, Result> {
89
90 public static class TableSnapshotRegionSplit extends InputSplit implements Writable {
91 private TableSnapshotInputFormatImpl.InputSplit delegate;
92
93
94 public TableSnapshotRegionSplit() {
95 this.delegate = new TableSnapshotInputFormatImpl.InputSplit();
96 }
97
98 public TableSnapshotRegionSplit(TableSnapshotInputFormatImpl.InputSplit delegate) {
99 this.delegate = delegate;
100 }
101
102 public TableSnapshotRegionSplit(HTableDescriptor htd, HRegionInfo regionInfo,
103 List<String> locations, Scan scan, Path restoreDir) {
104 this.delegate =
105 new TableSnapshotInputFormatImpl.InputSplit(htd, regionInfo, locations, scan, restoreDir);
106 }
107
108 @Override
109 public long getLength() throws IOException, InterruptedException {
110 return delegate.getLength();
111 }
112
113 @Override
114 public String[] getLocations() throws IOException, InterruptedException {
115 return delegate.getLocations();
116 }
117
118 @Override
119 public void write(DataOutput out) throws IOException {
120 delegate.write(out);
121 }
122
123 @Override
124 public void readFields(DataInput in) throws IOException {
125 delegate.readFields(in);
126 }
127
128 TableSnapshotInputFormatImpl.InputSplit getDelegate() {
129 return this.delegate;
130 }
131 }
132
133 @InterfaceAudience.Private
134 static class TableSnapshotRegionRecordReader extends
135 RecordReader<ImmutableBytesWritable, Result> {
136 private TableSnapshotInputFormatImpl.RecordReader delegate =
137 new TableSnapshotInputFormatImpl.RecordReader();
138 private TaskAttemptContext context;
139 private Method getCounter;
140
141 @Override
142 public void initialize(InputSplit split, TaskAttemptContext context) throws IOException,
143 InterruptedException {
144 this.context = context;
145 getCounter = TableRecordReaderImpl.retrieveGetCounterWithStringsParams(context);
146 delegate.initialize(
147 ((TableSnapshotRegionSplit) split).delegate,
148 context.getConfiguration());
149 }
150
151 @Override
152 public boolean nextKeyValue() throws IOException, InterruptedException {
153 boolean result = delegate.nextKeyValue();
154 if (result) {
155 ScanMetrics scanMetrics = delegate.getScanner().getScanMetrics();
156 if (scanMetrics != null && context != null) {
157 TableRecordReaderImpl.updateCounters(scanMetrics, 0, getCounter, context, 0);
158 }
159 }
160 return result;
161 }
162
163 @Override
164 public ImmutableBytesWritable getCurrentKey() throws IOException, InterruptedException {
165 return delegate.getCurrentKey();
166 }
167
168 @Override
169 public Result getCurrentValue() throws IOException, InterruptedException {
170 return delegate.getCurrentValue();
171 }
172
173 @Override
174 public float getProgress() throws IOException, InterruptedException {
175 return delegate.getProgress();
176 }
177
178 @Override
179 public void close() throws IOException {
180 delegate.close();
181 }
182 }
183
184 @Override
185 public RecordReader<ImmutableBytesWritable, Result> createRecordReader(
186 InputSplit split, TaskAttemptContext context) throws IOException {
187 return new TableSnapshotRegionRecordReader();
188 }
189
190 @Override
191 public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
192 List<InputSplit> results = new ArrayList<InputSplit>();
193 for (TableSnapshotInputFormatImpl.InputSplit split :
194 TableSnapshotInputFormatImpl.getSplits(job.getConfiguration())) {
195 results.add(new TableSnapshotRegionSplit(split));
196 }
197 return results;
198 }
199
200
201
202
203
204
205
206
207
208
209 public static void setInput(Job job, String snapshotName, Path restoreDir)
210 throws IOException {
211 TableSnapshotInputFormatImpl.setInput(job.getConfiguration(), snapshotName, restoreDir);
212 }
213
214
215
216
217
218
219
220
221
222
223
224
225 public static void setInput(Job job, String snapshotName, Path restoreDir,
226 RegionSplitter.SplitAlgorithm splitAlgo, int numSplitsPerRegion) throws IOException {
227 TableSnapshotInputFormatImpl.setInput(job.getConfiguration(), snapshotName, restoreDir,
228 splitAlgo, numSplitsPerRegion);
229 }
230 }