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 with small scan at " + this.currentRegion);
134 }
135 return false;
136 }
137 localStartKey = endKey;
138 if (LOG.isDebugEnabled()) {
139 LOG.debug("Finished with region " + 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 internal small scanner to startKey at '"
151 + Bytes.toStringBinary(localStartKey) + "'");
152 }
153 smallScanCallable = getSmallScanCallable(
154 scan, getConnection(), getTable(), localStartKey, cacheNum);
155 if (this.scanMetrics != null && skipRowOfFirstResult == null) {
156 this.scanMetrics.countOfRegions.incrementAndGet();
157 }
158 return true;
159 }
160
161 static RegionServerCallable<Result[]> getSmallScanCallable(
162 final Scan sc, HConnection connection, TableName table, byte[] localStartKey,
163 final int cacheNum) throws IOException {
164 sc.setStartRow(localStartKey);
165 RegionServerCallable<Result[]> callable = new RegionServerCallable<Result[]>(
166 connection, table, sc.getStartRow()) {
167 public Result[] call() throws IOException {
168 ScanRequest request = RequestConverter.buildScanRequest(getLocation()
169 .getRegionInfo().getRegionName(), sc, cacheNum, true);
170 ScanResponse response = null;
171 PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
172 try {
173 controller.setPriority(getTableName());
174 response = getStub().scan(controller, request);
175 return ResponseConverter.getResults(controller.cellScanner(),
176 response);
177 } catch (ServiceException se) {
178 throw ProtobufUtil.getRemoteException(se);
179 }
180 }
181 };
182 return callable;
183 }
184
185 @Override
186 public Result next() throws IOException {
187
188
189 if (cache.size() == 0 && this.closed) {
190 return null;
191 }
192 if (cache.size() == 0) {
193 Result[] values = null;
194 long remainingResultSize = maxScannerResultSize;
195 int countdown = this.caching;
196 boolean currentRegionDone = false;
197
198 while (remainingResultSize > 0 && countdown > 0
199 && nextScanner(countdown, values == null, currentRegionDone)) {
200
201
202
203 values = this.caller.callWithRetries(smallScanCallable);
204 this.currentRegion = smallScanCallable.getHRegionInfo();
205 long currentTime = System.currentTimeMillis();
206 if (this.scanMetrics != null) {
207 this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime
208 - lastNext);
209 }
210 lastNext = currentTime;
211 if (values != null && values.length > 0) {
212 for (int i = 0; i < values.length; i++) {
213 Result rs = values[i];
214 if (i == 0 && this.skipRowOfFirstResult != null
215 && Bytes.equals(skipRowOfFirstResult, rs.getRow())) {
216
217 continue;
218 }
219 cache.add(rs);
220 for (Cell kv : rs.rawCells()) {
221 remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
222 }
223 countdown--;
224 this.lastResult = rs;
225 }
226 }
227 currentRegionDone = countdown > 0;
228 }
229 }
230
231 if (cache.size() > 0) {
232 return cache.poll();
233 }
234
235
236 writeScanMetrics();
237 return null;
238 }
239
240 @Override
241 public void close() {
242 if (!scanMetricsPublished) writeScanMetrics();
243 closed = true;
244 }
245 }