001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.mapred; 019 020import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT; 021 022import java.io.IOException; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.hbase.DoNotRetryIOException; 025import org.apache.hadoop.hbase.client.Result; 026import org.apache.hadoop.hbase.client.ResultScanner; 027import org.apache.hadoop.hbase.client.Scan; 028import org.apache.hadoop.hbase.client.Table; 029import org.apache.hadoop.hbase.filter.Filter; 030import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 031import org.apache.hadoop.hbase.mapreduce.TableInputFormat; 032import org.apache.hadoop.hbase.util.Bytes; 033import org.apache.hadoop.util.StringUtils; 034import org.apache.yetus.audience.InterfaceAudience; 035import org.slf4j.Logger; 036import org.slf4j.LoggerFactory; 037 038/** 039 * Iterate over an HBase table data, return (Text, RowResult) pairs 040 */ 041@InterfaceAudience.Public 042public class TableRecordReaderImpl { 043 private static final Logger LOG = LoggerFactory.getLogger(TableRecordReaderImpl.class); 044 045 private byte[] startRow; 046 private byte[] endRow; 047 private byte[] lastSuccessfulRow; 048 private Filter trrRowFilter; 049 private ResultScanner scanner; 050 private Table htable; 051 private byte[][] trrInputColumns; 052 private long timestamp; 053 private int rowcount; 054 private boolean logScannerActivity = false; 055 private int logPerRowCount = 100; 056 057 /** 058 * Restart from survivable exceptions by creating a new scanner. 059 */ 060 public void restart(byte[] firstRow) throws IOException { 061 Scan currentScan; 062 if ((endRow != null) && (endRow.length > 0)) { 063 if (trrRowFilter != null) { 064 Scan scan = new Scan(firstRow, endRow); 065 TableInputFormat.addColumns(scan, trrInputColumns); 066 scan.setFilter(trrRowFilter); 067 scan.setCacheBlocks(false); 068 this.scanner = this.htable.getScanner(scan); 069 currentScan = scan; 070 } else { 071 LOG.debug("TIFB.restart, firstRow: " + Bytes.toStringBinary(firstRow) + ", endRow: " 072 + Bytes.toStringBinary(endRow)); 073 Scan scan = new Scan(firstRow, endRow); 074 TableInputFormat.addColumns(scan, trrInputColumns); 075 this.scanner = this.htable.getScanner(scan); 076 currentScan = scan; 077 } 078 } else { 079 LOG.debug("TIFB.restart, firstRow: " + Bytes.toStringBinary(firstRow) + ", no endRow"); 080 081 Scan scan = new Scan(firstRow); 082 TableInputFormat.addColumns(scan, trrInputColumns); 083 scan.setFilter(trrRowFilter); 084 this.scanner = this.htable.getScanner(scan); 085 currentScan = scan; 086 } 087 if (logScannerActivity) { 088 LOG.info("Current scan=" + currentScan.toString()); 089 timestamp = System.currentTimeMillis(); 090 rowcount = 0; 091 } 092 } 093 094 /** 095 * Build the scanner. Not done in constructor to allow for extension. 096 */ 097 public void init() throws IOException { 098 restart(startRow); 099 } 100 101 byte[] getStartRow() { 102 return this.startRow; 103 } 104 105 /** 106 * @param htable the {@link org.apache.hadoop.hbase.HTableDescriptor} to scan. 107 */ 108 public void setHTable(Table htable) { 109 Configuration conf = htable.getConfiguration(); 110 logScannerActivity = conf.getBoolean( 111 "hbase.client.log.scanner.activity" /* ScannerCallable.LOG_SCANNER_ACTIVITY */, false); 112 logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100); 113 this.htable = htable; 114 } 115 116 /** 117 * @param inputColumns the columns to be placed in {@link Result}. 118 */ 119 public void setInputColumns(final byte[][] inputColumns) { 120 this.trrInputColumns = inputColumns; 121 } 122 123 /** 124 * @param startRow the first row in the split 125 */ 126 public void setStartRow(final byte[] startRow) { 127 this.startRow = startRow; 128 } 129 130 /** 131 * @param endRow the last row in the split 132 */ 133 public void setEndRow(final byte[] endRow) { 134 this.endRow = endRow; 135 } 136 137 /** 138 * @param rowFilter the {@link Filter} to be used. 139 */ 140 public void setRowFilter(Filter rowFilter) { 141 this.trrRowFilter = rowFilter; 142 } 143 144 public void close() { 145 if (this.scanner != null) { 146 this.scanner.close(); 147 } 148 try { 149 this.htable.close(); 150 } catch (IOException ioe) { 151 LOG.warn("Error closing table", ioe); 152 } 153 } 154 155 /** 156 * @see org.apache.hadoop.mapred.RecordReader#createKey() 157 */ 158 public ImmutableBytesWritable createKey() { 159 return new ImmutableBytesWritable(); 160 } 161 162 /** 163 * @see org.apache.hadoop.mapred.RecordReader#createValue() 164 */ 165 public Result createValue() { 166 return new Result(); 167 } 168 169 public long getPos() { 170 // This should be the ordinal tuple in the range; 171 // not clear how to calculate... 172 return 0; 173 } 174 175 public float getProgress() { 176 // Depends on the total number of tuples and getPos 177 return 0; 178 } 179 180 /** 181 * @param key HStoreKey as input key. 182 * @param value MapWritable as input value 183 * @return true if there was more data 184 */ 185 public boolean next(ImmutableBytesWritable key, Result value) throws IOException { 186 Result result; 187 try { 188 try { 189 result = this.scanner.next(); 190 if (logScannerActivity) { 191 rowcount++; 192 if (rowcount >= logPerRowCount) { 193 long now = System.currentTimeMillis(); 194 LOG.info("Mapper took " + (now - timestamp) + "ms to process " + rowcount + " rows"); 195 timestamp = now; 196 rowcount = 0; 197 } 198 } 199 } catch (IOException e) { 200 // do not retry if the exception tells us not to do so 201 if (e instanceof DoNotRetryIOException) { 202 throw e; 203 } 204 // try to handle all other IOExceptions by restarting 205 // the scanner, if the second call fails, it will be rethrown 206 LOG.debug("recovered from " + StringUtils.stringifyException(e)); 207 if (lastSuccessfulRow == null) { 208 LOG.warn("We are restarting the first next() invocation," 209 + " if your mapper has restarted a few other times like this" 210 + " then you should consider killing this job and investigate" 211 + " why it's taking so long."); 212 } 213 if (lastSuccessfulRow == null) { 214 restart(startRow); 215 } else { 216 restart(lastSuccessfulRow); 217 this.scanner.next(); // skip presumed already mapped row 218 } 219 result = this.scanner.next(); 220 } 221 222 if (result != null && result.size() > 0) { 223 key.set(result.getRow()); 224 lastSuccessfulRow = key.get(); 225 value.copyFrom(result); 226 return true; 227 } 228 return false; 229 } catch (IOException ioe) { 230 if (logScannerActivity) { 231 long now = System.currentTimeMillis(); 232 LOG.info("Mapper took " + (now - timestamp) + "ms to process " + rowcount + " rows"); 233 LOG.info(ioe.toString(), ioe); 234 String lastRow = 235 lastSuccessfulRow == null ? "null" : Bytes.toStringBinary(lastSuccessfulRow); 236 LOG.info("lastSuccessfulRow=" + lastRow); 237 } 238 throw ioe; 239 } 240 } 241}