1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.mapreduce;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.List;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.classification.InterfaceAudience;
27 import org.apache.hadoop.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.client.HTable;
29 import org.apache.hadoop.hbase.client.Result;
30 import org.apache.hadoop.hbase.client.Scan;
31 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
32 import org.apache.hadoop.hbase.util.Bytes;
33 import org.apache.hadoop.hbase.util.Pair;
34 import org.apache.hadoop.mapreduce.InputFormat;
35 import org.apache.hadoop.mapreduce.InputSplit;
36 import org.apache.hadoop.mapreduce.JobContext;
37 import org.apache.hadoop.mapreduce.RecordReader;
38 import org.apache.hadoop.mapreduce.TaskAttemptContext;
39
40
41
42
43
44
45 @InterfaceAudience.Public
46 @InterfaceStability.Evolving
47 public abstract class MultiTableInputFormatBase extends
48 InputFormat<ImmutableBytesWritable, Result> {
49
50 final Log LOG = LogFactory.getLog(MultiTableInputFormatBase.class);
51
52
53 private List<Scan> scans;
54
55
56 private TableRecordReader tableRecordReader = null;
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71 @Override
72 public RecordReader<ImmutableBytesWritable, Result> createRecordReader(
73 InputSplit split, TaskAttemptContext context)
74 throws IOException, InterruptedException {
75 TableSplit tSplit = (TableSplit) split;
76
77 if (tSplit.getTableName() == null) {
78 throw new IOException("Cannot create a record reader because of a"
79 + " previous error. Please look at the previous logs lines from"
80 + " the task's full log for more details.");
81 }
82 HTable table =
83 new HTable(context.getConfiguration(), tSplit.getTableName());
84
85 TableRecordReader trr = this.tableRecordReader;
86
87 if (trr == null) {
88 trr = new TableRecordReader();
89 }
90 Scan sc = tSplit.getScan();
91 sc.setStartRow(tSplit.getStartRow());
92 sc.setStopRow(tSplit.getEndRow());
93 trr.setScan(sc);
94 trr.setHTable(table);
95 trr.initialize(split, context);
96 return trr;
97 }
98
99
100
101
102
103
104
105
106
107
108 @Override
109 public List<InputSplit> getSplits(JobContext context) throws IOException {
110 if (scans.isEmpty()) {
111 throw new IOException("No scans were provided.");
112 }
113 List<InputSplit> splits = new ArrayList<InputSplit>();
114
115 for (Scan scan : scans) {
116 byte[] tableName = scan.getAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME);
117 if (tableName == null)
118 throw new IOException("A scan object did not have a table name");
119 HTable table = new HTable(context.getConfiguration(), tableName);
120 Pair<byte[][], byte[][]> keys = table.getStartEndKeys();
121 if (keys == null || keys.getFirst() == null ||
122 keys.getFirst().length == 0) {
123 throw new IOException("Expecting at least one region for table : "
124 + Bytes.toString(tableName));
125 }
126 int count = 0;
127
128 byte[] startRow = scan.getStartRow();
129 byte[] stopRow = scan.getStopRow();
130
131 for (int i = 0; i < keys.getFirst().length; i++) {
132 if (!includeRegionInSplit(keys.getFirst()[i], keys.getSecond()[i])) {
133 continue;
134 }
135 String regionLocation =
136 table.getRegionLocation(keys.getFirst()[i], false).getHostname();
137
138
139 if ((startRow.length == 0 || keys.getSecond()[i].length == 0 ||
140 Bytes.compareTo(startRow, keys.getSecond()[i]) < 0) &&
141 (stopRow.length == 0 ||
142 Bytes.compareTo(stopRow, keys.getFirst()[i]) > 0)) {
143 byte[] splitStart =
144 startRow.length == 0 ||
145 Bytes.compareTo(keys.getFirst()[i], startRow) >= 0 ? keys
146 .getFirst()[i] : startRow;
147 byte[] splitStop =
148 (stopRow.length == 0 || Bytes.compareTo(keys.getSecond()[i],
149 stopRow) <= 0) && keys.getSecond()[i].length > 0 ? keys
150 .getSecond()[i] : stopRow;
151 InputSplit split =
152 new TableSplit(tableName, scan, splitStart,
153 splitStop, regionLocation);
154 splits.add(split);
155 if (LOG.isDebugEnabled())
156 LOG.debug("getSplits: split -> " + (count++) + " -> " + split);
157 }
158 }
159 table.close();
160 }
161 return splits;
162 }
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186 protected boolean includeRegionInSplit(final byte[] startKey,
187 final byte[] endKey) {
188 return true;
189 }
190
191
192
193
194 protected List<Scan> getScans() {
195 return this.scans;
196 }
197
198
199
200
201
202
203 protected void setScans(List<Scan> scans) {
204 this.scans = scans;
205 }
206
207
208
209
210
211
212
213 protected void setTableRecordReader(TableRecordReader tableRecordReader) {
214 this.tableRecordReader = tableRecordReader;
215 }
216 }