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