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.IOException;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.classification.InterfaceStability;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.client.HTable;
29  import org.apache.hadoop.hbase.client.Result;
30  import org.apache.hadoop.hbase.filter.Filter;
31  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
32  import org.apache.hadoop.mapred.InputFormat;
33  import org.apache.hadoop.mapred.InputSplit;
34  import org.apache.hadoop.mapred.JobConf;
35  import org.apache.hadoop.mapred.RecordReader;
36  import org.apache.hadoop.mapred.Reporter;
37  
38  /**
39   * A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a
40   * byte[] of input columns and optionally a {@link Filter}.
41   * Subclasses may use other TableRecordReader implementations.
42   * <p>
43   * An example of a subclass:
44   * <pre>
45   *   class ExampleTIF extends TableInputFormatBase implements JobConfigurable {
46   *
47   *     public void configure(JobConf job) {
48   *       HTable exampleTable = new HTable(HBaseConfiguration.create(job),
49   *         Bytes.toBytes("exampleTable"));
50   *       // mandatory
51   *       setHTable(exampleTable);
52   *       Text[] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
53   *         Bytes.toBytes("columnB") };
54   *       // mandatory
55   *       setInputColumns(inputColumns);
56   *       RowFilterInterface exampleFilter = new RegExpRowFilter("keyPrefix.*");
57   *       // optional
58   *       setRowFilter(exampleFilter);
59   *     }
60   *
61   *     public void validateInput(JobConf job) throws IOException {
62   *     }
63   *  }
64   * </pre>
65   */
66  
67  @InterfaceAudience.Public
68  @InterfaceStability.Stable
69  public abstract class TableInputFormatBase
70  implements InputFormat<ImmutableBytesWritable, Result> {
71    private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
72    private byte [][] inputColumns;
73    private HTable table;
74    private TableRecordReader tableRecordReader;
75    private Filter rowFilter;
76  
77    /**
78     * Builds a TableRecordReader. If no TableRecordReader was provided, uses
79     * the default.
80     *
81     * @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit,
82     *      JobConf, Reporter)
83     */
84    public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
85        InputSplit split, JobConf job, Reporter reporter)
86    throws IOException {
87      TableSplit tSplit = (TableSplit) split;
88      TableRecordReader trr = this.tableRecordReader;
89      // if no table record reader was provided use default
90      if (trr == null) {
91        trr = new TableRecordReader();
92      }
93      trr.setStartRow(tSplit.getStartRow());
94      trr.setEndRow(tSplit.getEndRow());
95      trr.setHTable(this.table);
96      trr.setInputColumns(this.inputColumns);
97      trr.setRowFilter(this.rowFilter);
98      trr.init();
99      return trr;
100   }
101 
102   /**
103    * Calculates the splits that will serve as input for the map tasks.
104    * <ul>
105    * Splits are created in number equal to the smallest between numSplits and
106    * the number of {@link org.apache.hadoop.hbase.regionserver.HRegion}s in the table. 
107    * If the number of splits is smaller than the number of 
108    * {@link org.apache.hadoop.hbase.regionserver.HRegion}s then splits are spanned across
109    * multiple {@link org.apache.hadoop.hbase.regionserver.HRegion}s 
110    * and are grouped the most evenly possible. In the
111    * case splits are uneven the bigger splits are placed first in the
112    * {@link InputSplit} array.
113    *
114    * @param job the map task {@link JobConf}
115    * @param numSplits a hint to calculate the number of splits (mapred.map.tasks).
116    *
117    * @return the input splits
118    *
119    * @see org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop.mapred.JobConf, int)
120    */
121   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
122     if (this.table == null) {
123       throw new IOException("No table was provided");
124     }
125     byte [][] startKeys = this.table.getStartKeys();
126     if (startKeys == null || startKeys.length == 0) {
127       throw new IOException("Expecting at least one region");
128     }
129     if (this.inputColumns == null || this.inputColumns.length == 0) {
130       throw new IOException("Expecting at least one column");
131     }
132     int realNumSplits = numSplits > startKeys.length? startKeys.length:
133       numSplits;
134     InputSplit[] splits = new InputSplit[realNumSplits];
135     int middle = startKeys.length / realNumSplits;
136     int startPos = 0;
137     for (int i = 0; i < realNumSplits; i++) {
138       int lastPos = startPos + middle;
139       lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
140       String regionLocation = table.getRegionLocation(startKeys[startPos]).
141         getHostname();
142       splits[i] = new TableSplit(this.table.getName(),
143         startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
144           HConstants.EMPTY_START_ROW, regionLocation);
145       LOG.info("split: " + i + "->" + splits[i]);
146       startPos = lastPos;
147     }
148     return splits;
149   }
150 
151   /**
152    * @param inputColumns to be passed in {@link Result} to the map task.
153    */
154   protected void setInputColumns(byte [][] inputColumns) {
155     this.inputColumns = inputColumns;
156   }
157 
158   /**
159    * Allows subclasses to get the {@link HTable}.
160    */
161   protected HTable getHTable() {
162     return this.table;
163   }
164 
165   /**
166    * Allows subclasses to set the {@link HTable}.
167    *
168    * @param table to get the data from
169    */
170   protected void setHTable(HTable table) {
171     this.table = table;
172   }
173 
174   /**
175    * Allows subclasses to set the {@link TableRecordReader}.
176    *
177    * @param tableRecordReader
178    *                to provide other {@link TableRecordReader} implementations.
179    */
180   protected void setTableRecordReader(TableRecordReader tableRecordReader) {
181     this.tableRecordReader = tableRecordReader;
182   }
183 
184   /**
185    * Allows subclasses to set the {@link Filter} to be used.
186    *
187    * @param rowFilter
188    */
189   protected void setRowFilter(Filter rowFilter) {
190     this.rowFilter = rowFilter;
191   }
192 }