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.protobuf.ServiceException;
44
45
46
47
48
49
50
51
52 @InterfaceAudience.Private
53 public class ClientSmallScanner extends ClientScanner {
54 private final Log LOG = LogFactory.getLog(this.getClass());
55 private ScannerCallableWithReplicas smallScanCallable = null;
56
57
58 private byte[] skipRowOfFirstResult = null;
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73 public ClientSmallScanner(final Configuration conf, final Scan scan,
74 final TableName tableName, ClusterConnection connection,
75 RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory,
76 ExecutorService pool, int primaryOperationTimeout) throws IOException {
77 super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
78 primaryOperationTimeout);
79 }
80
81 @Override
82 protected void initializeScannerInConstruction() throws IOException {
83
84
85 }
86
87
88
89
90
91
92
93
94
95
96 private boolean nextScanner(int nbRows, final boolean done,
97 boolean currentRegionDone) throws IOException {
98
99 byte[] localStartKey;
100 int cacheNum = nbRows;
101 skipRowOfFirstResult = null;
102
103 if (this.currentRegion != null && currentRegionDone) {
104 byte[] endKey = this.currentRegion.getEndKey();
105 if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
106 || checkScanStopRow(endKey) || done) {
107 close();
108 if (LOG.isDebugEnabled()) {
109 LOG.debug("Finished with small scan at " + this.currentRegion);
110 }
111 return false;
112 }
113 localStartKey = endKey;
114 if (LOG.isDebugEnabled()) {
115 LOG.debug("Finished with region " + this.currentRegion);
116 }
117 } else if (this.lastResult != null) {
118 localStartKey = this.lastResult.getRow();
119 skipRowOfFirstResult = this.lastResult.getRow();
120 cacheNum++;
121 } else {
122 localStartKey = this.scan.getStartRow();
123 }
124
125 if (LOG.isTraceEnabled()) {
126 LOG.trace("Advancing internal small scanner to startKey at '"
127 + Bytes.toStringBinary(localStartKey) + "'");
128 }
129 smallScanCallable = getSmallScanCallable(
130 getConnection(), getTable(), scan, getScanMetrics(), localStartKey, cacheNum,
131 rpcControllerFactory, getPool(), getPrimaryOperationTimeout(),
132 getRetries(), getScannerTimeout(), getConf(), caller);
133 if (this.scanMetrics != null && skipRowOfFirstResult == null) {
134 this.scanMetrics.countOfRegions.incrementAndGet();
135 }
136 return true;
137 }
138
139
140 static ScannerCallableWithReplicas getSmallScanCallable(
141 ClusterConnection connection, TableName table, Scan scan,
142 ScanMetrics scanMetrics, byte[] localStartKey, final int cacheNum,
143 RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout,
144 int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller<Result []> caller) {
145 scan.setStartRow(localStartKey);
146 SmallScannerCallable s = new SmallScannerCallable(
147 connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0);
148 ScannerCallableWithReplicas scannerCallableWithReplicas =
149 new ScannerCallableWithReplicas(table, connection,
150 s, pool, primaryOperationTimeout, scan, retries,
151 scannerTimeout, cacheNum, conf, caller);
152 return scannerCallableWithReplicas;
153 }
154
155 static class SmallScannerCallable extends ScannerCallable {
156 public SmallScannerCallable(
157 ClusterConnection connection, TableName table, Scan scan,
158 ScanMetrics scanMetrics, RpcControllerFactory controllerFactory, int caching, int id) {
159 super(connection, table, scan, scanMetrics, controllerFactory, id);
160 this.setCaching(caching);
161 }
162
163 @Override
164 public Result[] call(int timeout) throws IOException {
165 if (this.closed) return null;
166 if (Thread.interrupted()) {
167 throw new InterruptedIOException();
168 }
169 ScanRequest request = RequestConverter.buildScanRequest(getLocation()
170 .getRegionInfo().getRegionName(), getScan(), getCaching(), true);
171 ScanResponse response = null;
172 PayloadCarryingRpcController controller = controllerFactory.newController();
173 try {
174 controller.setPriority(getTableName());
175 controller.setCallTimeout(timeout);
176 response = getStub().scan(controller, request);
177 return ResponseConverter.getResults(controller.cellScanner(),
178 response);
179 } catch (ServiceException se) {
180 throw ProtobufUtil.getRemoteException(se);
181 }
182 }
183
184 @Override
185 public ScannerCallable getScannerCallableForReplica(int id) {
186 return new SmallScannerCallable((ClusterConnection)connection, tableName, getScan(), scanMetrics,
187 controllerFactory, getCaching(), id);
188 }
189 }
190
191 @Override
192 public Result next() throws IOException {
193
194
195 if (cache.size() == 0 && this.closed) {
196 return null;
197 }
198 if (cache.size() == 0) {
199 Result[] values = null;
200 long remainingResultSize = maxScannerResultSize;
201 int countdown = this.caching;
202 boolean currentRegionDone = false;
203
204 while (remainingResultSize > 0 && countdown > 0
205 && nextScanner(countdown, values == null, currentRegionDone)) {
206
207
208
209
210
211 values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout);
212 this.currentRegion = smallScanCallable.getHRegionInfo();
213 long currentTime = System.currentTimeMillis();
214 if (this.scanMetrics != null) {
215 this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
216 - lastNext);
217 }
218 lastNext = currentTime;
219 if (values != null && values.length > 0) {
220 for (int i = 0; i < values.length; i++) {
221 Result rs = values[i];
222 if (i == 0 && this.skipRowOfFirstResult != null
223 && Bytes.equals(skipRowOfFirstResult, rs.getRow())) {
224
225 continue;
226 }
227 cache.add(rs);
228
229 for (Cell cell : rs.rawCells()) {
230 remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
231 }
232 countdown--;
233 this.lastResult = rs;
234 }
235 }
236 currentRegionDone = countdown > 0;
237 }
238 }
239
240 if (cache.size() > 0) {
241 return cache.poll();
242 }
243
244
245 writeScanMetrics();
246 return null;
247 }
248
249 @Override
250 public void close() {
251 if (!scanMetricsPublished) writeScanMetrics();
252 closed = true;
253 }
254 }