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.conf.Configuration;
26  import org.apache.hadoop.hbase.client.HTable;
27  import org.apache.hadoop.hbase.client.Result;
28  import org.apache.hadoop.hbase.client.ResultScanner;
29  import org.apache.hadoop.hbase.client.Scan;
30  import org.apache.hadoop.hbase.client.ScannerCallable;
31  import org.apache.hadoop.hbase.filter.Filter;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
34  import org.apache.hadoop.hbase.util.Bytes;
35  import org.apache.hadoop.util.StringUtils;
36  
37  import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT;
38  
39  /**
40   * Iterate over an HBase table data, return (Text, RowResult) pairs
41   */
42  public class TableRecordReaderImpl {
43    static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
44  
45    private byte [] startRow;
46    private byte [] endRow;
47    private byte [] lastSuccessfulRow;
48    private Filter trrRowFilter;
49    private ResultScanner scanner;
50    private HTable htable;
51    private byte [][] trrInputColumns;
52    private long timestamp;
53    private int rowcount;
54    private boolean logScannerActivity = false;
55    private int logPerRowCount = 100;
56  
57    /**
58     * Restart from survivable exceptions by creating a new scanner.
59     *
60     * @param firstRow
61     * @throws IOException
62     */
63    public void restart(byte[] firstRow) throws IOException {
64      Scan currentScan;
65      if ((endRow != null) && (endRow.length > 0)) {
66        if (trrRowFilter != null) {
67          Scan scan = new Scan(firstRow, endRow);
68          TableInputFormat.addColumns(scan, trrInputColumns);
69          scan.setFilter(trrRowFilter);
70          scan.setCacheBlocks(false);
71          this.scanner = this.htable.getScanner(scan);
72          currentScan = scan;
73        } else {
74          LOG.debug("TIFB.restart, firstRow: " +
75              Bytes.toStringBinary(firstRow) + ", endRow: " +
76              Bytes.toStringBinary(endRow));
77          Scan scan = new Scan(firstRow, endRow);
78          TableInputFormat.addColumns(scan, trrInputColumns);
79          this.scanner = this.htable.getScanner(scan);
80          currentScan = scan;
81        }
82      } else {
83        LOG.debug("TIFB.restart, firstRow: " +
84            Bytes.toStringBinary(firstRow) + ", no endRow");
85  
86        Scan scan = new Scan(firstRow);
87        TableInputFormat.addColumns(scan, trrInputColumns);
88        scan.setFilter(trrRowFilter);
89        this.scanner = this.htable.getScanner(scan);
90        currentScan = scan;
91      }
92      if (logScannerActivity) {
93        LOG.info("Current scan=" + currentScan.toString());
94        timestamp = System.currentTimeMillis();
95        rowcount = 0;
96      }
97    }
98  
99    /**
100    * Build the scanner. Not done in constructor to allow for extension.
101    *
102    * @throws IOException
103    */
104   public void init() throws IOException {
105     restart(startRow);
106   }
107 
108   byte[] getStartRow() {
109     return this.startRow;
110   }
111   /**
112    * @param htable the {@link HTable} to scan.
113    */
114   public void setHTable(HTable htable) {
115     Configuration conf = htable.getConfiguration();
116     logScannerActivity = conf.getBoolean(
117       ScannerCallable.LOG_SCANNER_ACTIVITY, false);
118     logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
119     this.htable = htable;
120   }
121 
122   /**
123    * @param inputColumns the columns to be placed in {@link Result}.
124    */
125   public void setInputColumns(final byte [][] inputColumns) {
126     this.trrInputColumns = inputColumns;
127   }
128 
129   /**
130    * @param startRow the first row in the split
131    */
132   public void setStartRow(final byte [] startRow) {
133     this.startRow = startRow;
134   }
135 
136   /**
137    *
138    * @param endRow the last row in the split
139    */
140   public void setEndRow(final byte [] endRow) {
141     this.endRow = endRow;
142   }
143 
144   /**
145    * @param rowFilter the {@link Filter} to be used.
146    */
147   public void setRowFilter(Filter rowFilter) {
148     this.trrRowFilter = rowFilter;
149   }
150 
151   public void close() {
152     this.scanner.close();
153   }
154 
155   /**
156    * @return ImmutableBytesWritable
157    *
158    * @see org.apache.hadoop.mapred.RecordReader#createKey()
159    */
160   public ImmutableBytesWritable createKey() {
161     return new ImmutableBytesWritable();
162   }
163 
164   /**
165    * @return RowResult
166    *
167    * @see org.apache.hadoop.mapred.RecordReader#createValue()
168    */
169   public Result createValue() {
170     return new Result();
171   }
172 
173   public long getPos() {
174     // This should be the ordinal tuple in the range;
175     // not clear how to calculate...
176     return 0;
177   }
178 
179   public float getProgress() {
180     // Depends on the total number of tuples and getPos
181     return 0;
182   }
183 
184   /**
185    * @param key HStoreKey as input key.
186    * @param value MapWritable as input value
187    * @return true if there was more data
188    * @throws IOException
189    */
190   public boolean next(ImmutableBytesWritable key, Result value)
191   throws IOException {
192     Result result;
193     try {
194       try {
195         result = this.scanner.next();
196         if (logScannerActivity) {
197           rowcount ++;
198           if (rowcount >= logPerRowCount) {
199             long now = System.currentTimeMillis();
200             LOG.info("Mapper took " + (now-timestamp)
201               + "ms to process " + rowcount + " rows");
202             timestamp = now;
203             rowcount = 0;
204           }
205         }
206       } catch (IOException e) {
207         // try to handle all IOExceptions by restarting
208         // the scanner, if the second call fails, it will be rethrown
209         LOG.debug("recovered from " + StringUtils.stringifyException(e));
210         if (lastSuccessfulRow == null) {
211           LOG.warn("We are restarting the first next() invocation," +
212               " if your mapper has restarted a few other times like this" +
213               " then you should consider killing this job and investigate" +
214               " why it's taking so long.");
215         }
216         if (lastSuccessfulRow == null) {
217           restart(startRow);
218         } else {
219           restart(lastSuccessfulRow);
220           this.scanner.next();    // skip presumed already mapped row
221         }
222         result = this.scanner.next();
223       }
224 
225       if (result != null && result.size() > 0) {
226         key.set(result.getRow());
227         lastSuccessfulRow = key.get();
228         value.copyFrom(result);
229         return true;
230       }
231       return false;
232     } catch (IOException ioe) {
233       if (logScannerActivity) {
234         long now = System.currentTimeMillis();
235         LOG.info("Mapper took " + (now-timestamp)
236           + "ms to process " + rowcount + " rows");
237         LOG.info(ioe);
238         String lastRow = lastSuccessfulRow == null ?
239           "null" : Bytes.toStringBinary(lastSuccessfulRow);
240         LOG.info("lastSuccessfulRow=" + lastRow);
241       }
242       throw ioe;
243     }
244   }
245 }