1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
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.Cell;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.KeyValueUtil;
30 import org.apache.hadoop.hbase.TableName;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.hbase.classification.InterfaceStability;
33 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
34 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
35 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
36 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
37 import org.apache.hadoop.hbase.protobuf.RequestConverter;
38 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
39 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
40 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
41 import org.apache.hadoop.hbase.util.Bytes;
42
43 import com.google.common.annotations.VisibleForTesting;
44 import com.google.protobuf.ServiceException;
45
46
47
48
49
50
51
52
53 @InterfaceAudience.Public
54 @InterfaceStability.Evolving
55 public class ClientSmallScanner extends ClientScanner {
56 private final Log LOG = LogFactory.getLog(this.getClass());
57 private RegionServerCallable<Result[]> smallScanCallable = null;
58 private SmallScannerCallableFactory callableFactory;
59
60
61
62
63
64
65
66
67
68
69
70 public ClientSmallScanner(final Configuration conf, final Scan scan,
71 final TableName tableName) throws IOException {
72 this(conf, scan, tableName, HConnectionManager.getConnection(conf));
73 }
74
75
76
77
78
79
80
81
82
83
84
85 public ClientSmallScanner(final Configuration conf, final Scan scan,
86 final TableName tableName, HConnection connection) throws IOException {
87 this(conf, scan, tableName, connection, RpcRetryingCallerFactory.instantiate(conf,
88 connection.getStatisticsTracker()),
89 RpcControllerFactory.instantiate(conf));
90 }
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105 public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
106 HConnection connection, RpcRetryingCallerFactory rpcFactory,
107 RpcControllerFactory controllerFactory) throws IOException {
108 this(conf, scan, tableName, connection, rpcFactory, controllerFactory, new SmallScannerCallableFactory());
109 }
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132 @VisibleForTesting
133 ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName,
134 HConnection connection, RpcRetryingCallerFactory rpcFactory,
135 RpcControllerFactory controllerFactory, SmallScannerCallableFactory callableFactory)
136 throws IOException {
137 super(conf, scan, tableName, connection, rpcFactory, controllerFactory);
138 this.callableFactory = callableFactory;
139 }
140
141 @Override
142 protected void initializeScannerInConstruction() throws IOException {
143
144
145 }
146
147
148
149
150
151
152
153
154
155
156 private boolean nextScanner(int nbRows, final boolean done,
157 boolean currentRegionDone) throws IOException {
158
159 byte[] localStartKey;
160 int cacheNum = nbRows;
161 boolean regionChanged = true;
162
163 if (this.currentRegion != null && currentRegionDone) {
164 byte[] endKey = this.currentRegion.getEndKey();
165 if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
166 || checkScanStopRow(endKey) || done) {
167 close();
168 if (LOG.isDebugEnabled()) {
169 LOG.debug("Finished with small scan at " + this.currentRegion);
170 }
171 return false;
172 }
173 localStartKey = endKey;
174 if (LOG.isDebugEnabled()) {
175 LOG.debug("Finished with region " + this.currentRegion);
176 }
177 } else if (this.lastResult != null) {
178 regionChanged = false;
179 localStartKey = Bytes.add(lastResult.getRow(), new byte[1]);
180 } else {
181 localStartKey = this.scan.getStartRow();
182 }
183
184 if (LOG.isTraceEnabled()) {
185 LOG.trace("Advancing internal small scanner to startKey at '"
186 + Bytes.toStringBinary(localStartKey) + "'");
187 }
188 smallScanCallable = callableFactory.getCallable(
189 scan, getConnection(), getTable(), scanMetrics, localStartKey, cacheNum,
190 rpcControllerFactory);
191 if (this.scanMetrics != null && regionChanged) {
192 this.scanMetrics.countOfRegions.incrementAndGet();
193 }
194 return true;
195 }
196
197 static class SmallScannerCallable extends ScannerCallable {
198 public SmallScannerCallable(HConnection connection, TableName tableName, Scan scan,
199 ScanMetrics scanMetrics, PayloadCarryingRpcController controller, int cacheNum) {
200 super(connection, tableName, scan, scanMetrics, controller);
201 this.setCaching(cacheNum);
202 }
203
204 @Override
205 public Result [] call() throws IOException {
206 if (this.closed) return null;
207 if (Thread.interrupted()) {
208 throw new InterruptedIOException();
209 }
210 ScanRequest request = RequestConverter.buildScanRequest(getLocation()
211 .getRegionInfo().getRegionName(), getScan(), getCaching(), true);
212 ScanResponse response = null;
213 try {
214 controller.setPriority(getTableName());
215 response = getStub().scan(controller, request);
216 Result[] results = ResponseConverter.getResults(controller.cellScanner(), response);
217 if (response.hasMoreResultsInRegion()) {
218 setHasMoreResultsContext(true);
219 setServerHasMoreResults(response.getMoreResultsInRegion());
220 } else {
221 setHasMoreResultsContext(false);
222 }
223 updateResultsMetrics(results);
224 return results;
225 } catch (ServiceException se) {
226 throw ProtobufUtil.getRemoteException(se);
227 }
228 }
229 }
230
231 @Override
232 public Result next() throws IOException {
233
234
235 if (cache.size() == 0 && this.closed) {
236 return null;
237 }
238 if (cache.size() == 0) {
239 loadCache();
240 }
241
242 if (cache.size() > 0) {
243 return cache.poll();
244 }
245
246
247 writeScanMetrics();
248 return null;
249 }
250
251 @Override
252 protected void loadCache() throws IOException {
253 Result[] values = null;
254 long remainingResultSize = maxScannerResultSize;
255 int countdown = this.caching;
256 boolean currentRegionDone = false;
257
258 while (remainingResultSize > 0 && countdown > 0
259 && nextScanner(countdown, values == null, currentRegionDone)) {
260
261
262
263 values = this.caller.callWithRetries(smallScanCallable);
264 this.currentRegion = smallScanCallable.getHRegionInfo();
265 long currentTime = System.currentTimeMillis();
266 if (this.scanMetrics != null) {
267 this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
268 - lastNext);
269 }
270 lastNext = currentTime;
271 if (values != null && values.length > 0) {
272 for (int i = 0; i < values.length; i++) {
273 Result rs = values[i];
274 cache.add(rs);
275 for (Cell kv : rs.rawCells()) {
276 remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
277 }
278 countdown--;
279 this.lastResult = rs;
280 }
281 }
282 if (smallScanCallable.hasMoreResultsContext()) {
283
284 currentRegionDone = !smallScanCallable.getServerHasMoreResults();
285 } else {
286
287 currentRegionDone = countdown > 0;
288 }
289 }
290 }
291
292 public void close() {
293 if (!scanMetricsPublished) writeScanMetrics();
294 closed = true;
295 }
296
297 @VisibleForTesting
298 protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) {
299 this.callableFactory = callableFactory;
300 }
301
302 @VisibleForTesting
303 protected void setRpcRetryingCaller(RpcRetryingCaller<Result []> caller) {
304 this.caller = caller;
305 }
306
307 @VisibleForTesting
308 protected void setRpcControllerFactory(RpcControllerFactory rpcControllerFactory) {
309 this.rpcControllerFactory = rpcControllerFactory;
310 }
311
312 @InterfaceAudience.Private
313 protected static class SmallScannerCallableFactory {
314
315 public RegionServerCallable<Result[]> getCallable(final Scan sc, HConnection connection,
316 TableName tableName, ScanMetrics scanMetrics, byte[] localStartKey, final int cacheNum,
317 final RpcControllerFactory rpcControllerFactory) throws IOException {
318 sc.setStartRow(localStartKey);
319 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
320 return new SmallScannerCallable(connection, tableName, sc, scanMetrics, controller,
321 cacheNum);
322 }
323
324 }
325 }