1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.mapreduce;
19
20 import java.io.IOException;
21 import java.lang.reflect.Method;
22 import java.util.Map;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
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.Result;
30 import org.apache.hadoop.hbase.client.ResultScanner;
31 import org.apache.hadoop.hbase.client.Scan;
32 import org.apache.hadoop.hbase.client.ScannerCallable;
33 import org.apache.hadoop.hbase.client.Table;
34 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
35 import org.apache.hadoop.hbase.DoNotRetryIOException;
36 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.mapreduce.Counter;
39 import org.apache.hadoop.mapreduce.InputSplit;
40 import org.apache.hadoop.mapreduce.TaskAttemptContext;
41 import org.apache.hadoop.util.StringUtils;
42
43
44
45
46
47 @InterfaceAudience.Public
48 @InterfaceStability.Stable
49 public class TableRecordReaderImpl {
50 public static final String LOG_PER_ROW_COUNT
51 = "hbase.mapreduce.log.scanner.rowcount";
52
53 private static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
54
55
56 @InterfaceAudience.Private
57 static final String HBASE_COUNTER_GROUP_NAME = "HBase Counters";
58
59 private ResultScanner scanner = null;
60 private Scan scan = null;
61 private Scan currentScan = null;
62 private Table htable = null;
63 private byte[] lastSuccessfulRow = null;
64 private ImmutableBytesWritable key = null;
65 private Result value = null;
66 private TaskAttemptContext context = null;
67 private Method getCounter = null;
68 private long numRestarts = 0;
69 private long numStale = 0;
70 private long timestamp;
71 private int rowcount;
72 private boolean logScannerActivity = false;
73 private int logPerRowCount = 100;
74
75
76
77
78
79
80
81 public void restart(byte[] firstRow) throws IOException {
82 currentScan = new Scan(scan);
83 currentScan.withStartRow(firstRow);
84 currentScan.setScanMetricsEnabled(true);
85 if (this.scanner != null) {
86 if (logScannerActivity) {
87 LOG.info("Closing the previously opened scanner object.");
88 }
89 this.scanner.close();
90 }
91 this.scanner = this.htable.getScanner(currentScan);
92 if (logScannerActivity) {
93 LOG.info("Current scan=" + currentScan.toString());
94 timestamp = System.currentTimeMillis();
95 rowcount = 0;
96 }
97 }
98
99
100
101
102
103
104
105 protected static Method retrieveGetCounterWithStringsParams(TaskAttemptContext context)
106 throws IOException {
107 Method m = null;
108 try {
109 m = context.getClass().getMethod("getCounter",
110 new Class [] {String.class, String.class});
111 } catch (SecurityException e) {
112 throw new IOException("Failed test for getCounter", e);
113 } catch (NoSuchMethodException e) {
114
115 }
116 return m;
117 }
118
119
120
121
122
123
124 public void setHTable(Table htable) {
125 Configuration conf = htable.getConfiguration();
126 logScannerActivity = conf.getBoolean(
127 ScannerCallable.LOG_SCANNER_ACTIVITY, false);
128 logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
129 this.htable = htable;
130 }
131
132
133
134
135
136
137 public void setScan(Scan scan) {
138 this.scan = scan;
139 }
140
141
142
143
144
145
146
147 public void initialize(InputSplit inputsplit,
148 TaskAttemptContext context) throws IOException,
149 InterruptedException {
150 if (context != null) {
151 this.context = context;
152 getCounter = retrieveGetCounterWithStringsParams(context);
153 }
154 restart(scan.getStartRow());
155 }
156
157
158
159
160
161
162 public void close() {
163 if (this.scanner != null) {
164 this.scanner.close();
165 }
166 try {
167 this.htable.close();
168 } catch (IOException ioe) {
169 LOG.warn("Error closing table", ioe);
170 }
171 }
172
173
174
175
176
177
178
179
180 public ImmutableBytesWritable getCurrentKey() throws IOException,
181 InterruptedException {
182 return key;
183 }
184
185
186
187
188
189
190
191
192 public Result getCurrentValue() throws IOException, InterruptedException {
193 return value;
194 }
195
196
197
198
199
200
201
202
203
204 public boolean nextKeyValue() throws IOException, InterruptedException {
205 if (key == null) key = new ImmutableBytesWritable();
206 if (value == null) value = new Result();
207 try {
208 try {
209 value = this.scanner.next();
210 if (value != null && value.isStale()) numStale++;
211 if (logScannerActivity) {
212 rowcount ++;
213 if (rowcount >= logPerRowCount) {
214 long now = System.currentTimeMillis();
215 LOG.info("Mapper took " + (now-timestamp)
216 + "ms to process " + rowcount + " rows");
217 timestamp = now;
218 rowcount = 0;
219 }
220 }
221 } catch (IOException e) {
222
223 if (e instanceof DoNotRetryIOException) {
224 throw e;
225 }
226
227
228 LOG.info("recovered from " + StringUtils.stringifyException(e));
229 if (lastSuccessfulRow == null) {
230 LOG.warn("We are restarting the first next() invocation," +
231 " if your mapper has restarted a few other times like this" +
232 " then you should consider killing this job and investigate" +
233 " why it's taking so long.");
234 }
235 if (lastSuccessfulRow == null) {
236 restart(scan.getStartRow());
237 } else {
238 restart(lastSuccessfulRow);
239 scanner.next();
240 }
241 value = scanner.next();
242 if (value != null && value.isStale()) numStale++;
243 numRestarts++;
244 }
245
246 if (value != null && value.size() > 0) {
247 key.set(value.getRow());
248 lastSuccessfulRow = key.get();
249 return true;
250 }
251
252
253 if (value != null && value.isCursor()) {
254 key.set(value.getCursor().getRow());
255 lastSuccessfulRow = key.get();
256 return true;
257 }
258
259 updateCounters();
260 return false;
261 } catch (IOException ioe) {
262 if (logScannerActivity) {
263 long now = System.currentTimeMillis();
264 LOG.info("Mapper took " + (now-timestamp)
265 + "ms to process " + rowcount + " rows");
266 LOG.info(ioe);
267 String lastRow = lastSuccessfulRow == null ?
268 "null" : Bytes.toStringBinary(lastSuccessfulRow);
269 LOG.info("lastSuccessfulRow=" + lastRow);
270 }
271 throw ioe;
272 }
273 }
274
275
276
277
278
279
280
281
282 private void updateCounters() throws IOException {
283 ScanMetrics scanMetrics = scanner.getScanMetrics();
284 if (scanMetrics == null) {
285 return;
286 }
287
288 updateCounters(scanMetrics, numRestarts, getCounter, context, numStale);
289 }
290
291 protected static void updateCounters(ScanMetrics scanMetrics, long numScannerRestarts,
292 Method getCounter, TaskAttemptContext context, long numStale) {
293
294 if (getCounter == null) {
295 return;
296 }
297
298 try {
299 for (Map.Entry<String, Long> entry:scanMetrics.getMetricsMap().entrySet()) {
300 Counter ct = (Counter)getCounter.invoke(context,
301 HBASE_COUNTER_GROUP_NAME, entry.getKey());
302
303 ct.increment(entry.getValue());
304 }
305 ((Counter) getCounter.invoke(context, HBASE_COUNTER_GROUP_NAME,
306 "NUM_SCANNER_RESTARTS")).increment(numScannerRestarts);
307 ((Counter) getCounter.invoke(context, HBASE_COUNTER_GROUP_NAME,
308 "NUM_SCAN_RESULTS_STALE")).increment(numStale);
309 } catch (Exception e) {
310 LOG.debug("can't update counter." + StringUtils.stringifyException(e));
311 }
312 }
313
314
315
316
317
318
319 public float getProgress() {
320
321 return 0;
322 }
323
324 }