1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.util.Comparator;
24 import java.util.List;
25 import java.util.PriorityQueue;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.Cell;
30 import org.apache.hadoop.hbase.KeyValue.KVComparator;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
33
34
35
36
37
38
39
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
48 implements KeyValueScanner, InternalScanner {
49 private static final Log LOG = LogFactory.getLog(KeyValueHeap.class);
50 protected PriorityQueue<KeyValueScanner> heap = null;
51
52
53
54
55
56
57
58
59
60
61
62 protected KeyValueScanner current = null;
63
64 protected KVScannerComparator comparator;
65
66
67
68
69
70
71
72 public KeyValueHeap(List<? extends KeyValueScanner> scanners,
73 KVComparator comparator) throws IOException {
74 this(scanners, new KVScannerComparator(comparator));
75 }
76
77
78
79
80
81
82
83 KeyValueHeap(List<? extends KeyValueScanner> scanners,
84 KVScannerComparator comparator) throws IOException {
85 this.comparator = comparator;
86 if (!scanners.isEmpty()) {
87 this.heap = new PriorityQueue<KeyValueScanner>(scanners.size(),
88 this.comparator);
89 for (KeyValueScanner scanner : scanners) {
90 if (scanner.peek() != null) {
91 this.heap.add(scanner);
92 } else {
93 scanner.close();
94 }
95 }
96 this.current = pollRealKV();
97 }
98 }
99
100 public Cell peek() {
101 if (this.current == null) {
102 return null;
103 }
104 return this.current.peek();
105 }
106
107 public Cell next() throws IOException {
108 if(this.current == null) {
109 return null;
110 }
111 Cell kvReturn = this.current.next();
112 Cell kvNext = this.current.peek();
113 if (kvNext == null) {
114 this.current.close();
115 this.current = null;
116 this.current = pollRealKV();
117 } else {
118 KeyValueScanner topScanner = this.heap.peek();
119
120 if (topScanner != null && this.comparator.compare(kvNext, topScanner.peek()) >= 0) {
121 this.heap.add(this.current);
122 this.current = null;
123 this.current = pollRealKV();
124 }
125 }
126 return kvReturn;
127 }
128
129
130
131
132
133
134
135
136
137
138
139 @Override
140 public boolean next(List<Cell> result) throws IOException {
141 return next(result, NoLimitScannerContext.getInstance());
142 }
143
144 @Override
145 public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
146 if (this.current == null) {
147 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
148 }
149 InternalScanner currentAsInternal = (InternalScanner)this.current;
150 boolean moreCells = currentAsInternal.next(result, scannerContext);
151 Cell pee = this.current.peek();
152
153
154
155
156
157
158
159
160
161 if (pee == null || !moreCells) {
162 this.current.close();
163 } else {
164 this.heap.add(this.current);
165 }
166 this.current = null;
167 this.current = pollRealKV();
168 if (this.current == null) {
169 moreCells = scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
170 }
171 return moreCells;
172 }
173
174 protected static class KVScannerComparator implements Comparator<KeyValueScanner> {
175 protected KVComparator kvComparator;
176
177
178
179
180 public KVScannerComparator(KVComparator kvComparator) {
181 this.kvComparator = kvComparator;
182 }
183 public int compare(KeyValueScanner left, KeyValueScanner right) {
184 int comparison = compare(left.peek(), right.peek());
185 if (comparison != 0) {
186 return comparison;
187 } else {
188
189
190 long leftSequenceID = left.getSequenceID();
191 long rightSequenceID = right.getSequenceID();
192 if (leftSequenceID > rightSequenceID) {
193 return -1;
194 } else if (leftSequenceID < rightSequenceID) {
195 return 1;
196 } else {
197 return 0;
198 }
199 }
200 }
201
202
203
204
205
206
207 public int compare(Cell left, Cell right) {
208 return this.kvComparator.compare(left, right);
209 }
210
211
212
213 public KVComparator getComparator() {
214 return this.kvComparator;
215 }
216 }
217
218 public void close() {
219 if (this.current != null) {
220 this.current.close();
221 }
222 if (this.heap != null) {
223 KeyValueScanner scanner;
224 while ((scanner = this.heap.poll()) != null) {
225 scanner.close();
226 }
227 }
228 }
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245 @Override
246 public boolean seek(Cell seekKey) throws IOException {
247 return generalizedSeek(false,
248 seekKey,
249 false,
250 false);
251 }
252
253
254
255
256
257 @Override
258 public boolean reseek(Cell seekKey) throws IOException {
259 return generalizedSeek(false,
260 seekKey,
261 true,
262 false);
263 }
264
265
266
267
268 @Override
269 public boolean requestSeek(Cell key, boolean forward,
270 boolean useBloom) throws IOException {
271 return generalizedSeek(true, key, forward, useBloom);
272 }
273
274
275
276
277
278
279
280
281
282 private boolean generalizedSeek(boolean isLazy, Cell seekKey,
283 boolean forward, boolean useBloom) throws IOException {
284 if (!isLazy && useBloom) {
285 throw new IllegalArgumentException("Multi-column Bloom filter " +
286 "optimization requires a lazy seek");
287 }
288
289 if (current == null) {
290 return false;
291 }
292 heap.add(current);
293 current = null;
294
295 KeyValueScanner scanner = null;
296 try {
297 while ((scanner = heap.poll()) != null) {
298 Cell topKey = scanner.peek();
299 if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
300
301
302
303
304
305
306
307 heap.add(scanner);
308 scanner = null;
309 current = pollRealKV();
310 return current != null;
311 }
312
313 boolean seekResult;
314 if (isLazy && heap.size() > 0) {
315
316 seekResult = scanner.requestSeek(seekKey, forward, useBloom);
317 } else {
318 seekResult = NonLazyKeyValueScanner.doRealSeek(scanner, seekKey,
319 forward);
320 }
321
322 if (!seekResult) {
323 scanner.close();
324 } else {
325 heap.add(scanner);
326 }
327 }
328 } catch (Exception e) {
329 if (scanner != null) {
330 try {
331 scanner.close();
332 } catch (Exception ce) {
333 LOG.warn("close KeyValueScanner error", ce);
334 }
335 }
336 throw e;
337 }
338
339
340 return false;
341 }
342
343
344
345
346
347
348
349
350
351
352
353
354 protected KeyValueScanner pollRealKV() throws IOException {
355 KeyValueScanner kvScanner = heap.poll();
356 if (kvScanner == null) {
357 return null;
358 }
359
360 while (kvScanner != null && !kvScanner.realSeekDone()) {
361 if (kvScanner.peek() != null) {
362 try {
363 kvScanner.enforceSeek();
364 } catch (IOException ioe) {
365 kvScanner.close();
366 throw ioe;
367 }
368 Cell curKV = kvScanner.peek();
369 if (curKV != null) {
370 KeyValueScanner nextEarliestScanner = heap.peek();
371 if (nextEarliestScanner == null) {
372
373 return kvScanner;
374 }
375
376
377
378 Cell nextKV = nextEarliestScanner.peek();
379 if (nextKV == null || comparator.compare(curKV, nextKV) < 0) {
380
381 return kvScanner;
382 }
383
384
385
386
387 heap.add(kvScanner);
388 } else {
389
390
391 kvScanner.close();
392 }
393 } else {
394
395
396 kvScanner.close();
397 }
398 kvScanner = heap.poll();
399 }
400
401 return kvScanner;
402 }
403
404
405
406
407 public PriorityQueue<KeyValueScanner> getHeap() {
408 return this.heap;
409 }
410
411 @Override
412 public long getSequenceID() {
413 return 0;
414 }
415
416 KeyValueScanner getCurrentForTesting() {
417 return current;
418 }
419
420 @Override
421 public Cell getNextIndexedKey() {
422
423 return current == null ? null : current.getNextIndexedKey();
424 }
425 }