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.mapreduce;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.net.InetAddress;
24  import java.net.InetSocketAddress;
25  import java.net.UnknownHostException;
26  import java.util.ArrayList;
27  import java.util.HashMap;
28  import java.util.List;
29  
30  import javax.naming.NamingException;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.classification.InterfaceStability;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.HRegionLocation;
38  import org.apache.hadoop.hbase.TableName;
39  import org.apache.hadoop.hbase.client.Admin;
40  import org.apache.hadoop.hbase.client.Connection;
41  import org.apache.hadoop.hbase.client.ConnectionFactory;
42  import org.apache.hadoop.hbase.client.HTable;
43  import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
44  import org.apache.hadoop.hbase.client.RegionLocator;
45  import org.apache.hadoop.hbase.client.Result;
46  import org.apache.hadoop.hbase.client.Scan;
47  import org.apache.hadoop.hbase.client.Table;
48  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
49  import org.apache.hadoop.hbase.util.Addressing;
50  import org.apache.hadoop.hbase.util.Bytes;
51  import org.apache.hadoop.hbase.util.Pair;
52  import org.apache.hadoop.hbase.util.RegionSizeCalculator;
53  import org.apache.hadoop.hbase.util.Strings;
54  import org.apache.hadoop.mapreduce.InputFormat;
55  import org.apache.hadoop.mapreduce.InputSplit;
56  import org.apache.hadoop.mapreduce.JobContext;
57  import org.apache.hadoop.mapreduce.RecordReader;
58  import org.apache.hadoop.mapreduce.TaskAttemptContext;
59  import org.apache.hadoop.net.DNS;
60  import org.apache.hadoop.util.StringUtils;
61  
62  /**
63   * A base for {@link TableInputFormat}s. Receives a {@link Connection}, a {@link TableName},
64   * an {@link Scan} instance that defines the input columns etc. Subclasses may use
65   * other TableRecordReader implementations.
66   *
67   * Subclasses MUST ensure initializeTable(Connection, TableName) is called for an instance to
68   * function properly. Each of the entry points to this class used by the MapReduce framework,
69   * {@link #createRecordReader(InputSplit, TaskAttemptContext)} and {@link #getSplits(JobContext)},
70   * will call {@link #initialize(JobContext)} as a convenient centralized location to handle
71   * retrieving the necessary configuration information. If your subclass overrides either of these
72   * methods, either call the parent version or call initialize yourself.
73   *
74   * <p>
75   * An example of a subclass:
76   * <pre>
77   *   class ExampleTIF extends TableInputFormatBase {
78   *
79   *     {@literal @}Override
80   *     protected void initialize(JobContext context) throws IOException {
81   *       // We are responsible for the lifecycle of this connection until we hand it over in
82   *       // initializeTable.
83   *       Connection connection = ConnectionFactory.createConnection(HBaseConfiguration.create(
84   *              job.getConfiguration()));
85   *       TableName tableName = TableName.valueOf("exampleTable");
86   *       // mandatory. once passed here, TableInputFormatBase will handle closing the connection.
87   *       initializeTable(connection, tableName);
88   *       byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
89   *         Bytes.toBytes("columnB") };
90   *       // optional, by default we'll get everything for the table.
91   *       Scan scan = new Scan();
92   *       for (byte[] family : inputColumns) {
93   *         scan.addFamily(family);
94   *       }
95   *       Filter exampleFilter = new RowFilter(CompareOp.EQUAL, new RegexStringComparator("aa.*"));
96   *       scan.setFilter(exampleFilter);
97   *       setScan(scan);
98   *     }
99   *   }
100  * </pre>
101  */
102 @InterfaceAudience.Public
103 @InterfaceStability.Stable
104 public abstract class TableInputFormatBase
105 extends InputFormat<ImmutableBytesWritable, Result> {
106 
107   final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
108 
109   private static final String NOT_INITIALIZED = "The input format instance has not been properly " +
110       "initialized. Ensure you call initializeTable either in your constructor or initialize " +
111       "method";
112   private static final String INITIALIZATION_ERROR = "Cannot create a record reader because of a" +
113             " previous error. Please look at the previous logs lines from" +
114             " the task's full log for more details.";
115 
116   /** Holds the details for the internal scanner.
117    *
118    * @see Scan */
119   private Scan scan = null;
120   /** The {@link Admin}. */
121   private Admin admin;
122   /** The {@link Table} to scan. */
123   private Table table;
124   /** The {@link RegionLocator} of the table. */
125   private RegionLocator regionLocator;
126   /** The reader scanning the table, can be a custom one. */
127   private TableRecordReader tableRecordReader = null;
128   /** The underlying {@link Connection} of the table. */
129   private Connection connection;
130 
131   
132   /** The reverse DNS lookup cache mapping: IPAddress => HostName */
133   private HashMap<InetAddress, String> reverseDNSCacheMap =
134     new HashMap<InetAddress, String>();
135 
136   /**
137    * Builds a {@link TableRecordReader}. If no {@link TableRecordReader} was provided, uses
138    * the default.
139    *
140    * @param split  The split to work with.
141    * @param context  The current context.
142    * @return The newly created record reader.
143    * @throws IOException When creating the reader fails.
144    * @see org.apache.hadoop.mapreduce.InputFormat#createRecordReader(
145    *   org.apache.hadoop.mapreduce.InputSplit,
146    *   org.apache.hadoop.mapreduce.TaskAttemptContext)
147    */
148   @Override
149   public RecordReader<ImmutableBytesWritable, Result> createRecordReader(
150       InputSplit split, TaskAttemptContext context)
151   throws IOException {
152     // Just in case a subclass is relying on JobConfigurable magic.
153     if (table == null) {
154       initialize(context);
155     }
156     // null check in case our child overrides getTable to not throw.
157     try {
158       if (getTable() == null) {
159         // initialize() must not have been implemented in the subclass.
160         throw new IOException(INITIALIZATION_ERROR);
161       }
162     } catch (IllegalStateException exception) {
163       throw new IOException(INITIALIZATION_ERROR, exception);
164     }
165     TableSplit tSplit = (TableSplit) split;
166     LOG.info("Input split length: " + StringUtils.humanReadableInt(tSplit.getLength()) + " bytes.");
167     final TableRecordReader trr =
168         this.tableRecordReader != null ? this.tableRecordReader : new TableRecordReader();
169     Scan sc = new Scan(this.scan);
170     sc.setStartRow(tSplit.getStartRow());
171     sc.setStopRow(tSplit.getEndRow());
172     trr.setScan(sc);
173     trr.setTable(getTable());
174     return new RecordReader<ImmutableBytesWritable, Result>() {
175 
176       @Override
177       public void close() throws IOException {
178         trr.close();
179         closeTable();
180       }
181 
182       @Override
183       public ImmutableBytesWritable getCurrentKey() throws IOException, InterruptedException {
184         return trr.getCurrentKey();
185       }
186 
187       @Override
188       public Result getCurrentValue() throws IOException, InterruptedException {
189         return trr.getCurrentValue();
190       }
191 
192       @Override
193       public float getProgress() throws IOException, InterruptedException {
194         return trr.getProgress();
195       }
196 
197       @Override
198       public void initialize(InputSplit inputsplit, TaskAttemptContext context) throws IOException,
199           InterruptedException {
200         trr.initialize(inputsplit, context);
201       }
202 
203       @Override
204       public boolean nextKeyValue() throws IOException, InterruptedException {
205         return trr.nextKeyValue();
206       }
207     };
208   }
209 
210   protected Pair<byte[][],byte[][]> getStartEndKeys() throws IOException {
211     return getRegionLocator().getStartEndKeys();
212   }
213 
214   /**
215    * Calculates the splits that will serve as input for the map tasks. The
216    * number of splits matches the number of regions in a table.
217    *
218    * @param context  The current job context.
219    * @return The list of input splits.
220    * @throws IOException When creating the list of splits fails.
221    * @see org.apache.hadoop.mapreduce.InputFormat#getSplits(
222    *   org.apache.hadoop.mapreduce.JobContext)
223    */
224   @Override
225   public List<InputSplit> getSplits(JobContext context) throws IOException {
226     boolean closeOnFinish = false;
227 
228     // Just in case a subclass is relying on JobConfigurable magic.
229     if (table == null) {
230       initialize(context);
231       closeOnFinish = true;
232     }
233 
234     // null check in case our child overrides getTable to not throw.
235     try {
236       if (getTable() == null) {
237         // initialize() must not have been implemented in the subclass.
238         throw new IOException(INITIALIZATION_ERROR);
239       }
240     } catch (IllegalStateException exception) {
241       throw new IOException(INITIALIZATION_ERROR, exception);
242     }
243 
244     try {
245     RegionSizeCalculator sizeCalculator = new RegionSizeCalculator(regionLocator, admin);
246 
247     Pair<byte[][], byte[][]> keys = getStartEndKeys();
248     if (keys == null || keys.getFirst() == null ||
249         keys.getFirst().length == 0) {
250       HRegionLocation regLoc = regionLocator.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY, false);
251       if (null == regLoc) {
252         throw new IOException("Expecting at least one region.");
253       }
254       List<InputSplit> splits = new ArrayList<InputSplit>(1);
255       long regionSize = sizeCalculator.getRegionSize(regLoc.getRegionInfo().getRegionName());
256       TableSplit split = new TableSplit(table.getName(),
257           HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc
258               .getHostnamePort().split(Addressing.HOSTNAME_PORT_SEPARATOR)[0], regionSize);
259       splits.add(split);
260       return splits;
261     }
262     List<InputSplit> splits = new ArrayList<InputSplit>(keys.getFirst().length);
263     for (int i = 0; i < keys.getFirst().length; i++) {
264       if ( !includeRegionInSplit(keys.getFirst()[i], keys.getSecond()[i])) {
265         continue;
266       }
267       HRegionLocation location = regionLocator.getRegionLocation(keys.getFirst()[i], false);
268       // The below InetSocketAddress creation does a name resolution.
269       InetSocketAddress isa = new InetSocketAddress(location.getHostname(), location.getPort());
270       if (isa.isUnresolved()) {
271         LOG.warn("Failed resolve " + isa);
272       }
273       InetAddress regionAddress = isa.getAddress();
274       String regionLocation;
275       try {
276         regionLocation = reverseDNS(regionAddress);
277       } catch (NamingException e) {
278         LOG.warn("Cannot resolve the host name for " + regionAddress + " because of " + e);
279         regionLocation = location.getHostname();
280       }
281 
282       byte[] startRow = scan.getStartRow();
283       byte[] stopRow = scan.getStopRow();
284       // determine if the given start an stop key fall into the region
285       if ((startRow.length == 0 || keys.getSecond()[i].length == 0 ||
286           Bytes.compareTo(startRow, keys.getSecond()[i]) < 0) &&
287           (stopRow.length == 0 ||
288            Bytes.compareTo(stopRow, keys.getFirst()[i]) > 0)) {
289         byte[] splitStart = startRow.length == 0 ||
290           Bytes.compareTo(keys.getFirst()[i], startRow) >= 0 ?
291             keys.getFirst()[i] : startRow;
292         byte[] splitStop = (stopRow.length == 0 ||
293           Bytes.compareTo(keys.getSecond()[i], stopRow) <= 0) &&
294           keys.getSecond()[i].length > 0 ?
295             keys.getSecond()[i] : stopRow;
296 
297         byte[] regionName = location.getRegionInfo().getRegionName();
298         long regionSize = sizeCalculator.getRegionSize(regionName);
299         TableSplit split = new TableSplit(table.getName(),
300           splitStart, splitStop, regionLocation, regionSize);
301         splits.add(split);
302         if (LOG.isDebugEnabled()) {
303           LOG.debug("getSplits: split -> " + i + " -> " + split);
304         }
305       }
306     }
307     return splits;
308     } finally {
309       if (closeOnFinish) {
310         closeTable();
311       }
312     }
313   }
314 
315   /**
316    * @deprecated mistakenly made public in 0.98.7. scope will change to package-private
317    */
318   @Deprecated
319   public String reverseDNS(InetAddress ipAddress) throws NamingException, UnknownHostException {
320     String hostName = this.reverseDNSCacheMap.get(ipAddress);
321     if (hostName == null) {
322       String ipAddressString = null;
323       try {
324         ipAddressString = DNS.reverseDns(ipAddress, null);
325       } catch (Exception e) {
326         // We can use InetAddress in case the jndi failed to pull up the reverse DNS entry from the
327         // name service. Also, in case of ipv6, we need to use the InetAddress since resolving
328         // reverse DNS using jndi doesn't work well with ipv6 addresses.
329         ipAddressString = InetAddress.getByName(ipAddress.getHostAddress()).getHostName();
330       }
331       if (ipAddressString == null) throw new UnknownHostException("No host found for " + ipAddress);
332       hostName = Strings.domainNamePointerToHostName(ipAddressString);
333       this.reverseDNSCacheMap.put(ipAddress, hostName);
334     }
335     return hostName;
336   }
337 
338   /**
339    *
340    *
341    * Test if the given region is to be included in the InputSplit while splitting
342    * the regions of a table.
343    * <p>
344    * This optimization is effective when there is a specific reasoning to exclude an entire region from the M-R job,
345    * (and hence, not contributing to the InputSplit), given the start and end keys of the same. <br>
346    * Useful when we need to remember the last-processed top record and revisit the [last, current) interval for M-R processing,
347    * continuously. In addition to reducing InputSplits, reduces the load on the region server as well, due to the ordering of the keys.
348    * <br>
349    * <br>
350    * Note: It is possible that <code>endKey.length() == 0 </code> , for the last (recent) region.
351    * <br>
352    * Override this method, if you want to bulk exclude regions altogether from M-R. By default, no region is excluded( i.e. all regions are included).
353    *
354    *
355    * @param startKey Start key of the region
356    * @param endKey End key of the region
357    * @return true, if this region needs to be included as part of the input (default).
358    *
359    */
360   protected boolean includeRegionInSplit(final byte[] startKey, final byte [] endKey) {
361     return true;
362   }
363 
364   /**
365    * Allows subclasses to get the {@link HTable}.
366    *
367    * @deprecated use {@link #getTable()}
368    */
369   @Deprecated
370   protected HTable getHTable() {
371     return (HTable) this.getTable();
372   }
373 
374   /**
375    * Allows subclasses to get the {@link RegionLocator}.
376    */
377   protected RegionLocator getRegionLocator() {
378     if (regionLocator == null) {
379       throw new IllegalStateException(NOT_INITIALIZED);
380     }
381     return regionLocator;
382   }
383   
384   /**
385    * Allows subclasses to get the {@link Table}.
386    */
387   protected Table getTable() {
388     if (table == null) {
389       throw new IllegalStateException(NOT_INITIALIZED);
390     }
391     return table;
392   }
393 
394   /**
395    * Allows subclasses to get the {@link Admin}.
396    */
397   protected Admin getAdmin() {
398     if (admin == null) {
399       throw new IllegalStateException(NOT_INITIALIZED);
400     }
401     return admin;
402   }
403 
404   /**
405    * Allows subclasses to set the {@link HTable}.
406    *
407    * Will attempt to reuse the underlying Connection for our own needs, including
408    * retreiving an Admin interface to the HBase cluster.
409    *
410    * @param table  The table to get the data from.
411    * @throws IOException 
412    * @deprecated Use {@link #initializeTable(Connection, TableName)} instead.
413    */
414   @Deprecated
415   protected void setHTable(HTable table) throws IOException {
416     this.table = table;
417     this.connection = table.getConnection();
418     try {
419       this.regionLocator = table;
420       this.admin = this.connection.getAdmin();
421     } catch (NeedUnmanagedConnectionException exception) {
422       LOG.warn("You are using an HTable instance that relies on an HBase-managed Connection. " +
423           "This is usually due to directly creating an HTable, which is deprecated. Instead, you " +
424           "should create a Connection object and then request a Table instance from it. If you " +
425           "don't need the Table instance for your own use, you should instead use the " +
426           "TableInputFormatBase.initalizeTable method directly.");
427       LOG.info("Creating an additional unmanaged connection because user provided one can't be " +
428           "used for administrative actions. We'll close it when we close out the table.");
429       LOG.debug("Details about our failure to request an administrative interface.", exception);
430       // Do we need a "copy the settings from this Connection" method? are things like the User
431       // properly maintained by just looking again at the Configuration?
432       this.connection = ConnectionFactory.createConnection(this.connection.getConfiguration());
433       this.regionLocator = this.connection.getRegionLocator(table.getName());
434       this.admin = this.connection.getAdmin();
435     }
436   }
437 
438   /**
439    * Allows subclasses to initialize the table information.
440    *
441    * @param connection  The {@link Connection} to the HBase cluster. MUST be unmanaged. We will close.
442    * @param tableName  The {@link TableName} of the table to process. 
443    * @throws IOException 
444    */
445   protected void initializeTable(Connection connection, TableName tableName) throws IOException {
446     if (table != null || connection != null) {
447       LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
448           "reference; TableInputFormatBase will not close these old references when done.");
449     }
450     this.table = connection.getTable(tableName);
451     this.regionLocator = connection.getRegionLocator(tableName);
452     this.admin = connection.getAdmin();
453     this.connection = connection;
454   }
455 
456   /**
457    * Gets the scan defining the actual details like columns etc.
458    *
459    * @return The internal scan instance.
460    */
461   public Scan getScan() {
462     if (this.scan == null) this.scan = new Scan();
463     return scan;
464   }
465 
466   /**
467    * Sets the scan defining the actual details like columns etc.
468    *
469    * @param scan  The scan to set.
470    */
471   public void setScan(Scan scan) {
472     this.scan = scan;
473   }
474 
475   /**
476    * Allows subclasses to set the {@link TableRecordReader}.
477    *
478    * @param tableRecordReader A different {@link TableRecordReader}
479    *   implementation.
480    */
481   protected void setTableRecordReader(TableRecordReader tableRecordReader) {
482     this.tableRecordReader = tableRecordReader;
483   }
484   
485   /**
486    * Handle subclass specific set up.
487    * Each of the entry points used by the MapReduce framework,
488    * {@link #createRecordReader(InputSplit, TaskAttemptContext)} and {@link #getSplits(JobContext)},
489    * will call {@link #initialize(JobContext)} as a convenient centralized location to handle
490    * retrieving the necessary configuration information and calling
491    * {@link #initializeTable(Connection, TableName)}.
492    *
493    * Subclasses should implement their initialize call such that it is safe to call multiple times.
494    * The current TableInputFormatBase implementation relies on a non-null table reference to decide
495    * if an initialize call is needed, but this behavior may change in the future. In particular,
496    * it is critical that initializeTable not be called multiple times since this will leak
497    * Connection instances.
498    *
499    */
500   protected void initialize(JobContext context) throws IOException {
501   }
502 
503   /**
504    * Close the Table and related objects that were initialized via
505    * {@link #initializeTable(Connection, TableName)}.
506    *
507    * @throws IOException
508    */
509   protected void closeTable() throws IOException {
510     close(admin, table, regionLocator, connection);
511     admin = null;
512     table = null;
513     regionLocator = null;
514     connection = null;
515   }
516 
517   private void close(Closeable... closables) throws IOException {
518     for (Closeable c : closables) {
519       if(c != null) { c.close(); }
520     }
521   }
522 
523 }