1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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.DoNotRetryIOException;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.classification.InterfaceStability;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.client.HTable;
30 import org.apache.hadoop.hbase.client.Result;
31 import org.apache.hadoop.hbase.client.ResultScanner;
32 import org.apache.hadoop.hbase.client.Scan;
33 import org.apache.hadoop.hbase.client.ScannerCallable;
34 import org.apache.hadoop.hbase.filter.Filter;
35 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
36 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.util.StringUtils;
39
40 import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT;
41
42
43
44
45 @Deprecated
46 @InterfaceAudience.Public
47 @InterfaceStability.Stable
48 public class TableRecordReaderImpl {
49 static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
50
51 private byte [] startRow;
52 private byte [] endRow;
53 private byte [] lastSuccessfulRow;
54 private Filter trrRowFilter;
55 private ResultScanner scanner;
56 private HTable htable;
57 private byte [][] trrInputColumns;
58 private long timestamp;
59 private int rowcount;
60 private boolean logScannerActivity = false;
61 private int logPerRowCount = 100;
62
63
64
65
66
67
68
69 public void restart(byte[] firstRow) throws IOException {
70 Scan currentScan;
71 if ((endRow != null) && (endRow.length > 0)) {
72 if (trrRowFilter != null) {
73 Scan scan = new Scan(firstRow, endRow);
74 TableInputFormat.addColumns(scan, trrInputColumns);
75 scan.setFilter(trrRowFilter);
76 scan.setCacheBlocks(false);
77 this.scanner = this.htable.getScanner(scan);
78 currentScan = scan;
79 } else {
80 LOG.debug("TIFB.restart, firstRow: " +
81 Bytes.toStringBinary(firstRow) + ", endRow: " +
82 Bytes.toStringBinary(endRow));
83 Scan scan = new Scan(firstRow, endRow);
84 TableInputFormat.addColumns(scan, trrInputColumns);
85 this.scanner = this.htable.getScanner(scan);
86 currentScan = scan;
87 }
88 } else {
89 LOG.debug("TIFB.restart, firstRow: " +
90 Bytes.toStringBinary(firstRow) + ", no endRow");
91
92 Scan scan = new Scan(firstRow);
93 TableInputFormat.addColumns(scan, trrInputColumns);
94 scan.setFilter(trrRowFilter);
95 this.scanner = this.htable.getScanner(scan);
96 currentScan = scan;
97 }
98 if (logScannerActivity) {
99 LOG.info("Current scan=" + currentScan.toString());
100 timestamp = System.currentTimeMillis();
101 rowcount = 0;
102 }
103 }
104
105
106
107
108
109
110 public void init() throws IOException {
111 restart(startRow);
112 }
113
114 byte[] getStartRow() {
115 return this.startRow;
116 }
117
118
119
120 public void setHTable(HTable htable) {
121 Configuration conf = htable.getConfiguration();
122 logScannerActivity = conf.getBoolean(
123 ScannerCallable.LOG_SCANNER_ACTIVITY, false);
124 logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
125 this.htable = htable;
126 }
127
128
129
130
131 public void setInputColumns(final byte [][] inputColumns) {
132 this.trrInputColumns = inputColumns;
133 }
134
135
136
137
138 public void setStartRow(final byte [] startRow) {
139 this.startRow = startRow;
140 }
141
142
143
144
145
146 public void setEndRow(final byte [] endRow) {
147 this.endRow = endRow;
148 }
149
150
151
152
153 public void setRowFilter(Filter rowFilter) {
154 this.trrRowFilter = rowFilter;
155 }
156
157 public void close() {
158 this.scanner.close();
159 try {
160 this.htable.close();
161 } catch (IOException ioe) {
162 LOG.warn("Error closing table", ioe);
163 }
164 }
165
166
167
168
169
170
171 public ImmutableBytesWritable createKey() {
172 return new ImmutableBytesWritable();
173 }
174
175
176
177
178
179
180 public Result createValue() {
181 return new Result();
182 }
183
184 public long getPos() {
185
186
187 return 0;
188 }
189
190 public float getProgress() {
191
192 return 0;
193 }
194
195
196
197
198
199
200
201 public boolean next(ImmutableBytesWritable key, Result value)
202 throws IOException {
203 Result result;
204 try {
205 try {
206 result = this.scanner.next();
207 if (logScannerActivity) {
208 rowcount ++;
209 if (rowcount >= logPerRowCount) {
210 long now = System.currentTimeMillis();
211 LOG.info("Mapper took " + (now-timestamp)
212 + "ms to process " + rowcount + " rows");
213 timestamp = now;
214 rowcount = 0;
215 }
216 }
217 } catch (IOException e) {
218
219 if (e instanceof DoNotRetryIOException) {
220 throw e;
221 }
222
223
224 LOG.debug("recovered from " + StringUtils.stringifyException(e));
225 if (lastSuccessfulRow == null) {
226 LOG.warn("We are restarting the first next() invocation," +
227 " if your mapper has restarted a few other times like this" +
228 " then you should consider killing this job and investigate" +
229 " why it's taking so long.");
230 }
231 if (lastSuccessfulRow == null) {
232 restart(startRow);
233 } else {
234 restart(lastSuccessfulRow);
235 this.scanner.next();
236 }
237 result = this.scanner.next();
238 }
239
240 if (result != null && result.size() > 0) {
241 key.set(result.getRow());
242 lastSuccessfulRow = key.get();
243 value.copyFrom(result);
244 return true;
245 }
246 return false;
247 } catch (IOException ioe) {
248 if (logScannerActivity) {
249 long now = System.currentTimeMillis();
250 LOG.info("Mapper took " + (now-timestamp)
251 + "ms to process " + rowcount + " rows");
252 LOG.info(ioe);
253 String lastRow = lastSuccessfulRow == null ?
254 "null" : Bytes.toStringBinary(lastSuccessfulRow);
255 LOG.info("lastSuccessfulRow=" + lastRow);
256 }
257 throw ioe;
258 }
259 }
260 }