1 /**
2 *
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
10 *
11 * http://www.apache.org/licenses/LICENSE-2.0
12 *
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
18 */
19 package org.apache.hadoop.hbase.mapred;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.client.Connection;
31 import org.apache.hadoop.hbase.client.HTable;
32 import org.apache.hadoop.hbase.client.Result;
33 import org.apache.hadoop.hbase.client.Table;
34 import org.apache.hadoop.hbase.filter.Filter;
35 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
36 import org.apache.hadoop.mapred.InputFormat;
37 import org.apache.hadoop.mapred.InputSplit;
38 import org.apache.hadoop.mapred.JobConf;
39 import org.apache.hadoop.mapred.RecordReader;
40 import org.apache.hadoop.mapred.Reporter;
41
42 /**
43 * A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a
44 * byte[] of input columns and optionally a {@link Filter}.
45 * Subclasses may use other TableRecordReader implementations.
46 *
47 * Subclasses MUST ensure initializeTable(Connection, TableName) is called for an instance to
48 * function properly. Each of the entry points to this class used by the MapReduce framework,
49 * {@link #getRecordReader(InputSplit, JobConf, Reporter)} and {@link #getSplits(JobConf, int)},
50 * will call {@link #initialize(JobConf)} as a convenient centralized location to handle
51 * retrieving the necessary configuration information. If your subclass overrides either of these
52 * methods, either call the parent version or call initialize yourself.
53 *
54 * <p>
55 * An example of a subclass:
56 * <pre>
57 * class ExampleTIF extends TableInputFormatBase {
58 *
59 * {@literal @}Override
60 * protected void initialize(JobConf context) throws IOException {
61 * // We are responsible for the lifecycle of this connection until we hand it over in
62 * // initializeTable.
63 * Connection connection =
64 * ConnectionFactory.createConnection(HBaseConfiguration.create(job));
65 * TableName tableName = TableName.valueOf("exampleTable");
66 * // mandatory. once passed here, TableInputFormatBase will handle closing the connection.
67 * initializeTable(connection, tableName);
68 * byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
69 * Bytes.toBytes("columnB") };
70 * // mandatory
71 * setInputColumns(inputColumns);
72 * // optional, by default we'll get everything for the given columns.
73 * Filter exampleFilter = new RowFilter(CompareOp.EQUAL, new RegexStringComparator("aa.*"));
74 * setRowFilter(exampleFilter);
75 * }
76 * }
77 * </pre>
78 */
79
80 @InterfaceAudience.Public
81 @InterfaceStability.Stable
82 public abstract class TableInputFormatBase
83 implements InputFormat<ImmutableBytesWritable, Result> {
84 private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
85 private byte [][] inputColumns;
86 private HTable table;
87 private Connection connection;
88 private TableRecordReader tableRecordReader;
89 private Filter rowFilter;
90
91 private static final String NOT_INITIALIZED = "The input format instance has not been properly " +
92 "initialized. Ensure you call initializeTable either in your constructor or initialize " +
93 "method";
94 private static final String INITIALIZATION_ERROR = "Cannot create a record reader because of a" +
95 " previous error. Please look at the previous logs lines from" +
96 " the task's full log for more details.";
97
98 /**
99 * Builds a TableRecordReader. If no TableRecordReader was provided, uses
100 * the default.
101 *
102 * @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit,
103 * JobConf, Reporter)
104 */
105 @Override
106 public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
107 InputSplit split, JobConf job, Reporter reporter)
108 throws IOException {
109 // In case a subclass uses the deprecated approach or calls initializeTable directly
110 if (table == null) {
111 initialize(job);
112 }
113 // null check in case our child overrides getTable to not throw.
114 try {
115 if (getTable() == null) {
116 // initialize() must not have been implemented in the subclass.
117 throw new IOException(INITIALIZATION_ERROR);
118 }
119 } catch (IllegalStateException exception) {
120 throw new IOException(INITIALIZATION_ERROR, exception);
121 }
122
123 TableSplit tSplit = (TableSplit) split;
124 // if no table record reader was provided use default
125 final TableRecordReader trr = this.tableRecordReader == null ? new TableRecordReader() :
126 this.tableRecordReader;
127 trr.setStartRow(tSplit.getStartRow());
128 trr.setEndRow(tSplit.getEndRow());
129 trr.setHTable(this.table);
130 trr.setInputColumns(this.inputColumns);
131 trr.setRowFilter(this.rowFilter);
132 trr.init();
133 return new RecordReader<ImmutableBytesWritable, Result>() {
134
135 @Override
136 public void close() throws IOException {
137 trr.close();
138 closeTable();
139 }
140
141 @Override
142 public ImmutableBytesWritable createKey() {
143 return trr.createKey();
144 }
145
146 @Override
147 public Result createValue() {
148 return trr.createValue();
149 }
150
151 @Override
152 public long getPos() throws IOException {
153 return trr.getPos();
154 }
155
156 @Override
157 public float getProgress() throws IOException {
158 return trr.getProgress();
159 }
160
161 @Override
162 public boolean next(ImmutableBytesWritable key, Result value) throws IOException {
163 return trr.next(key, value);
164 }
165 };
166 }
167
168 /**
169 * Calculates the splits that will serve as input for the map tasks.
170 *
171 * Splits are created in number equal to the smallest between numSplits and
172 * the number of {@link org.apache.hadoop.hbase.regionserver.HRegion}s
173 * in the table. If the number of splits is smaller than the number of
174 * {@link org.apache.hadoop.hbase.regionserver.HRegion}s then splits are
175 * spanned across multiple {@link org.apache.hadoop.hbase.regionserver.HRegion}s
176 * and are grouped the most evenly possible. In the case splits are uneven the
177 * bigger splits are placed first in the {@link InputSplit} array.
178 *
179 * @param job the map task {@link JobConf}
180 * @param numSplits a hint to calculate the number of splits (mapred.map.tasks).
181 *
182 * @return the input splits
183 *
184 * @see org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop.mapred.JobConf, int)
185 */
186 @Override
187 public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
188 if (this.table == null) {
189 initialize(job);
190 }
191 // null check in case our child overrides getTable to not throw.
192 try {
193 if (getTable() == null) {
194 // initialize() must not have been implemented in the subclass.
195 throw new IOException(INITIALIZATION_ERROR);
196 }
197 } catch (IllegalStateException exception) {
198 throw new IOException(INITIALIZATION_ERROR, exception);
199 }
200
201 byte [][] startKeys = this.table.getStartKeys();
202 if (startKeys == null || startKeys.length == 0) {
203 throw new IOException("Expecting at least one region");
204 }
205 if (this.inputColumns == null || this.inputColumns.length == 0) {
206 throw new IOException("Expecting at least one column");
207 }
208 int realNumSplits = numSplits > startKeys.length? startKeys.length:
209 numSplits;
210 InputSplit[] splits = new InputSplit[realNumSplits];
211 int middle = startKeys.length / realNumSplits;
212 int startPos = 0;
213 for (int i = 0; i < realNumSplits; i++) {
214 int lastPos = startPos + middle;
215 lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
216 String regionLocation = table.getRegionLocation(startKeys[startPos]).
217 getHostname();
218 splits[i] = new TableSplit(this.table.getName(),
219 startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
220 HConstants.EMPTY_START_ROW, regionLocation);
221 LOG.info("split: " + i + "->" + splits[i]);
222 startPos = lastPos;
223 }
224 return splits;
225 }
226
227 /**
228 * Allows subclasses to initialize the table information.
229 *
230 * @param connection The Connection to the HBase cluster. MUST be unmanaged. We will close.
231 * @param tableName The {@link TableName} of the table to process.
232 * @throws IOException
233 */
234 protected void initializeTable(Connection connection, TableName tableName) throws IOException {
235 if (this.table != null || this.connection != null) {
236 LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
237 "reference; TableInputFormatBase will not close these old references when done.");
238 }
239 this.table = (HTable) connection.getTable(tableName);
240 this.connection = connection;
241 }
242
243 /**
244 * @param inputColumns to be passed in {@link Result} to the map task.
245 */
246 protected void setInputColumns(byte [][] inputColumns) {
247 this.inputColumns = inputColumns;
248 }
249
250 /**
251 * Allows subclasses to get the {@link HTable}.
252 * @deprecated use {@link #getTable()}
253 */
254 @Deprecated
255 protected HTable getHTable() {
256 return (HTable) getTable();
257 }
258
259 /**
260 * Allows subclasses to get the {@link Table}.
261 */
262 protected Table getTable() {
263 if (table == null) {
264 throw new IllegalStateException(NOT_INITIALIZED);
265 }
266 return this.table;
267 }
268
269 /**
270 * Allows subclasses to set the {@link HTable}.
271 *
272 * @param table to get the data from
273 * @deprecated use {@link #initializeTable(Connection,TableName)}
274 */
275 @Deprecated
276 protected void setHTable(HTable table) {
277 this.table = table;
278 }
279
280 /**
281 * Allows subclasses to set the {@link TableRecordReader}.
282 *
283 * @param tableRecordReader
284 * to provide other {@link TableRecordReader} implementations.
285 */
286 protected void setTableRecordReader(TableRecordReader tableRecordReader) {
287 this.tableRecordReader = tableRecordReader;
288 }
289
290 /**
291 * Allows subclasses to set the {@link Filter} to be used.
292 *
293 * @param rowFilter
294 */
295 protected void setRowFilter(Filter rowFilter) {
296 this.rowFilter = rowFilter;
297 }
298
299 /**
300 * Handle subclass specific set up.
301 * Each of the entry points used by the MapReduce framework,
302 * {@link #getRecordReader(InputSplit, JobConf, Reporter)} and {@link #getSplits(JobConf, int)},
303 * will call {@link #initialize(JobConf)} as a convenient centralized location to handle
304 * retrieving the necessary configuration information and calling
305 * {@link #initializeTable(Connection, TableName)}.
306 *
307 * Subclasses should implement their initialize call such that it is safe to call multiple times.
308 * The current TableInputFormatBase implementation relies on a non-null table reference to decide
309 * if an initialize call is needed, but this behavior may change in the future. In particular,
310 * it is critical that initializeTable not be called multiple times since this will leak
311 * Connection instances.
312 *
313 */
314 protected void initialize(JobConf job) throws IOException {
315 }
316
317 /**
318 * Close the Table and related objects that were initialized via
319 * {@link #initializeTable(Connection, TableName)}.
320 *
321 * @throws IOException
322 */
323 protected void closeTable() throws IOException {
324 close(table, connection);
325 table = null;
326 connection = null;
327 }
328
329 private void close(Closeable... closables) throws IOException {
330 for (Closeable c : closables) {
331 if(c != null) { c.close(); }
332 }
333 }
334 }