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