View Javadoc

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   public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
106       InputSplit split, JobConf job, Reporter reporter)
107   throws IOException {
108     // In case a subclass uses the deprecated approach or calls initializeTable directly
109     if (table == null) {
110       initialize(job);
111     }
112     // null check in case our child overrides getTable to not throw.
113     try {
114       if (getTable() == null) {
115         // initialize() must not have been implemented in the subclass.
116         throw new IOException(INITIALIZATION_ERROR);
117       }
118     } catch (IllegalStateException exception) {
119       throw new IOException(INITIALIZATION_ERROR, exception);
120     }
121 
122     TableSplit tSplit = (TableSplit) split;
123     // if no table record reader was provided use default
124     final TableRecordReader trr = this.tableRecordReader == null ? new TableRecordReader() :
125         this.tableRecordReader;
126     trr.setStartRow(tSplit.getStartRow());
127     trr.setEndRow(tSplit.getEndRow());
128     trr.setHTable(this.table);
129     trr.setInputColumns(this.inputColumns);
130     trr.setRowFilter(this.rowFilter);
131     trr.init();
132     return new RecordReader<ImmutableBytesWritable, Result>() {
133 
134       @Override
135       public void close() throws IOException {
136         trr.close();
137         closeTable();
138       }
139 
140       @Override
141       public ImmutableBytesWritable createKey() {
142         return trr.createKey();
143       }
144 
145       @Override
146       public Result createValue() {
147         return trr.createValue();
148       }
149 
150       @Override
151       public long getPos() throws IOException {
152         return trr.getPos();
153       }
154 
155       @Override
156       public float getProgress() throws IOException {
157         return trr.getProgress();
158       }
159 
160       @Override
161       public boolean next(ImmutableBytesWritable key, Result value) throws IOException {
162         return trr.next(key, value);
163       }
164     };
165   }
166 
167   /**
168    * Calculates the splits that will serve as input for the map tasks.
169    * <ul>
170    * Splits are created in number equal to the smallest between numSplits and
171    * the number of {@link org.apache.hadoop.hbase.regionserver.HRegion}s in the table. 
172    * If the number of splits is smaller than the number of 
173    * {@link org.apache.hadoop.hbase.regionserver.HRegion}s then splits are spanned across
174    * multiple {@link org.apache.hadoop.hbase.regionserver.HRegion}s 
175    * and are grouped the most evenly possible. In the
176    * case splits are uneven the bigger splits are placed first in the
177    * {@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   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
187     if (this.table == null) {
188       initialize(job);
189     }
190     // null check in case our child overrides getTable to not throw.
191     try {
192       if (getTable() == null) {
193         // initialize() must not have been implemented in the subclass.
194         throw new IOException(INITIALIZATION_ERROR);
195       }
196     } catch (IllegalStateException exception) {
197       throw new IOException(INITIALIZATION_ERROR, exception);
198     }
199 
200     byte [][] startKeys = this.table.getStartKeys();
201     if (startKeys == null || startKeys.length == 0) {
202       throw new IOException("Expecting at least one region");
203     }
204     if (this.inputColumns == null || this.inputColumns.length == 0) {
205       throw new IOException("Expecting at least one column");
206     }
207     int realNumSplits = numSplits > startKeys.length? startKeys.length:
208       numSplits;
209     InputSplit[] splits = new InputSplit[realNumSplits];
210     int middle = startKeys.length / realNumSplits;
211     int startPos = 0;
212     for (int i = 0; i < realNumSplits; i++) {
213       int lastPos = startPos + middle;
214       lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
215       String regionLocation = table.getRegionLocation(startKeys[startPos]).
216         getHostname();
217       splits[i] = new TableSplit(this.table.getName(),
218         startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
219           HConstants.EMPTY_START_ROW, regionLocation);
220       LOG.info("split: " + i + "->" + splits[i]);
221       startPos = lastPos;
222     }
223     return splits;
224   }
225 
226   /**
227    * Allows subclasses to initialize the table information.
228    *
229    * @param connection  The Connection to the HBase cluster. MUST be unmanaged. We will close.
230    * @param tableName  The {@link TableName} of the table to process.
231    * @throws IOException
232    */
233   protected void initializeTable(Connection connection, TableName tableName) throws IOException {
234     if (this.table != null || this.connection != null) {
235       LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
236           "reference; TableInputFormatBase will not close these old references when done.");
237     }
238     this.table = (HTable) connection.getTable(tableName);
239     this.connection = connection;
240   }
241 
242   /**
243    * @param inputColumns to be passed in {@link Result} to the map task.
244    */
245   protected void setInputColumns(byte [][] inputColumns) {
246     this.inputColumns = inputColumns;
247   }
248 
249   /**
250    * Allows subclasses to get the {@link HTable}.
251    * @deprecated use {@link #getTable()}
252    */
253   @Deprecated
254   protected HTable getHTable() {
255     return (HTable) getTable();
256   }
257 
258   /**
259    * Allows subclasses to get the {@link Table}.
260    */
261   protected Table getTable() {
262     if (table == null) {
263       throw new IllegalStateException(NOT_INITIALIZED);
264     }
265     return this.table;
266   }
267 
268   /**
269    * Allows subclasses to set the {@link HTable}.
270    *
271    * @param table to get the data from
272    * @deprecated use {@link #initializeTable(Connection,TableName)}
273    */
274   @Deprecated
275   protected void setHTable(HTable table) {
276     this.table = table;
277   }
278 
279   /**
280    * Allows subclasses to set the {@link TableRecordReader}.
281    *
282    * @param tableRecordReader
283    *                to provide other {@link TableRecordReader} implementations.
284    */
285   protected void setTableRecordReader(TableRecordReader tableRecordReader) {
286     this.tableRecordReader = tableRecordReader;
287   }
288 
289   /**
290    * Allows subclasses to set the {@link Filter} to be used.
291    *
292    * @param rowFilter
293    */
294   protected void setRowFilter(Filter rowFilter) {
295     this.rowFilter = rowFilter;
296   }
297 
298   /**
299    * Handle subclass specific set up.
300    * Each of the entry points used by the MapReduce framework,
301    * {@link #getRecordReader(InputSplit, JobConf, Reporter)} and {@link #getSplits(JobConf, int)},
302    * will call {@link #initialize(JobConf)} as a convenient centralized location to handle
303    * retrieving the necessary configuration information and calling
304    * {@link #initializeTable(Connection, TableName)}.
305    *
306    * Subclasses should implement their initialize call such that it is safe to call multiple times.
307    * The current TableInputFormatBase implementation relies on a non-null table reference to decide
308    * if an initialize call is needed, but this behavior may change in the future. In particular,
309    * it is critical that initializeTable not be called multiple times since this will leak
310    * Connection instances.
311    *
312    */
313   protected void initialize(JobConf job) throws IOException {
314   }
315 
316   /**
317    * Close the Table and related objects that were initialized via
318    * {@link #initializeTable(Connection, TableName)}.
319    *
320    * @throws IOException
321    */
322   protected void closeTable() throws IOException {
323     close(table, connection);
324     table = null;
325     connection = null;
326   }
327 
328   private void close(Closeable... closables) throws IOException {
329     for (Closeable c : closables) {
330       if(c != null) { c.close(); }
331     }
332   }
333 }