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
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.Cell;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.KeyValueUtil;
31 import org.apache.hadoop.hbase.TableName;
32 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
33 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34 import org.apache.hadoop.hbase.protobuf.RequestConverter;
35 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
36 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
37 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
38 import org.apache.hadoop.hbase.util.Bytes;
39
40 import com.google.protobuf.ServiceException;
41
42
43
44
45
46
47
48
49 @InterfaceAudience.Public
50 @InterfaceStability.Evolving
51 public class ClientSmallScanner extends ClientScanner {
52 private final Log LOG = LogFactory.getLog(this.getClass());
53 private RegionServerCallable<Result[]> smallScanCallable = null;
54
55
56 private byte[] skipRowOfFirstResult = null;
57
58
59
60
61
62
63
64
65
66
67
68 public ClientSmallScanner(final Configuration conf, final Scan scan,
69 final TableName tableName) throws IOException {
70 this(conf, scan, tableName, HConnectionManager.getConnection(conf));
71 }
72
73
74
75
76
77
78
79
80
81
82
83 public ClientSmallScanner(final Configuration conf, final Scan scan,
84 final TableName tableName, HConnection connection) throws IOException {
85 this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
86 }
87
88
89
90
91
92
93
94
95
96
97
98
99 public ClientSmallScanner(final Configuration conf, final Scan scan,
100 final TableName tableName, HConnection connection,
101 RpcRetryingCallerFactory rpcFactory) throws IOException {
102 super(conf, scan, tableName, connection, rpcFactory);
103 }
104
105 @Override
106 protected void initializeScannerInConstruction() throws IOException {
107
108
109 }
110
111
112
113
114
115
116
117
118
119
120 private boolean nextScanner(int nbRows, final boolean done,
121 boolean currentRegionDone) throws IOException {
122
123 byte[] localStartKey;
124 int cacheNum = nbRows;
125 skipRowOfFirstResult = null;
126
127 if (this.currentRegion != null && currentRegionDone) {
128 byte[] endKey = this.currentRegion.getEndKey();
129 if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
130 || checkScanStopRow(endKey) || done) {
131 close();
132 if (LOG.isDebugEnabled()) {
133 LOG.debug("Finished scan of " + this.currentRegion);
134 }
135 return false;
136 }
137 localStartKey = endKey;
138 if (LOG.isDebugEnabled()) {
139 LOG.debug("Finished with " + this.currentRegion);
140 }
141 } else if (this.lastResult != null) {
142 localStartKey = this.lastResult.getRow();
143 skipRowOfFirstResult = this.lastResult.getRow();
144 cacheNum++;
145 } else {
146 localStartKey = this.scan.getStartRow();
147 }
148
149 if (LOG.isTraceEnabled()) {
150 LOG.trace("Advancing to startKey at '" + Bytes.toStringBinary(localStartKey) + "'");
151 }
152 smallScanCallable = getSmallScanCallable(localStartKey, cacheNum);
153 if (this.scanMetrics != null && skipRowOfFirstResult == null) {
154 this.scanMetrics.countOfRegions.incrementAndGet();
155 }
156 return true;
157 }
158
159 private RegionServerCallable<Result[]> getSmallScanCallable(
160 byte[] localStartKey, final int cacheNum) {
161 this.scan.setStartRow(localStartKey);
162 RegionServerCallable<Result[]> callable = new RegionServerCallable<Result[]>(
163 getConnection(), getTable(), scan.getStartRow()) {
164 public Result[] call() throws IOException {
165 ScanRequest request = RequestConverter.buildScanRequest(getLocation()
166 .getRegionInfo().getRegionName(), scan, cacheNum, true);
167 ScanResponse response = null;
168 PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
169 try {
170 controller.setPriority(getTableName());
171 response = getStub().scan(controller, request);
172 return ResponseConverter.getResults(controller.cellScanner(),
173 response);
174 } catch (ServiceException se) {
175 throw ProtobufUtil.getRemoteException(se);
176 }
177 }
178 };
179 return callable;
180 }
181
182 @Override
183 public Result next() throws IOException {
184
185
186 if (cache.size() == 0 && this.closed) {
187 return null;
188 }
189 if (cache.size() == 0) {
190 Result[] values = null;
191 long remainingResultSize = maxScannerResultSize;
192 int countdown = this.caching;
193 boolean currentRegionDone = false;
194
195 while (remainingResultSize > 0 && countdown > 0
196 && nextScanner(countdown, values == null, currentRegionDone)) {
197
198
199
200 values = this.caller.callWithRetries(smallScanCallable);
201 this.currentRegion = smallScanCallable.getHRegionInfo();
202 long currentTime = System.currentTimeMillis();
203 if (this.scanMetrics != null) {
204 this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
205 - lastNext);
206 }
207 lastNext = currentTime;
208 if (values != null && values.length > 0) {
209 for (int i = 0; i < values.length; i++) {
210 Result rs = values[i];
211 if (i == 0 && this.skipRowOfFirstResult != null
212 && Bytes.equals(skipRowOfFirstResult, rs.getRow())) {
213
214 continue;
215 }
216 cache.add(rs);
217 for (Cell kv : rs.rawCells()) {
218 remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
219 }
220 countdown--;
221 this.lastResult = rs;
222 }
223 }
224 currentRegionDone = countdown > 0;
225 }
226 }
227
228 if (cache.size() > 0) {
229 return cache.poll();
230 }
231
232
233 writeScanMetrics();
234 return null;
235 }
236
237 @Override
238 public void close() {
239 if (!scanMetricsPublished) writeScanMetrics();
240 closed = true;
241 }
242 }