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