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