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