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