1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
19
20 import java.util.List;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.classification.InterfaceStability;
27 import org.apache.hadoop.hbase.client.metrics.ServerSideScanMetrics;
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
52 @InterfaceStability.Evolving
53 public class ScannerContext {
54 private static final Log LOG = LogFactory.getLog(ScannerContext.class);
55
56
57
58
59 LimitFields limits;
60 LimitFields progress;
61
62
63
64
65
66 NextState scannerState;
67 private static final NextState DEFAULT_STATE = NextState.MORE_VALUES;
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97 boolean keepProgress;
98 private static boolean DEFAULT_KEEP_PROGRESS = false;
99
100
101
102
103 final ServerSideScanMetrics metrics;
104
105 ScannerContext(boolean keepProgress, LimitFields limitsToCopy, boolean trackMetrics) {
106 this.limits = new LimitFields();
107 if (limitsToCopy != null) this.limits.copy(limitsToCopy);
108
109
110 progress = new LimitFields(0, LimitFields.DEFAULT_SCOPE, 0, LimitFields.DEFAULT_SCOPE, 0);
111
112 this.keepProgress = keepProgress;
113 this.scannerState = DEFAULT_STATE;
114 this.metrics = trackMetrics ? new ServerSideScanMetrics() : null;
115 }
116
117 boolean isTrackingMetrics() {
118 return this.metrics != null;
119 }
120
121
122
123
124
125
126 ServerSideScanMetrics getMetrics() {
127 assert isTrackingMetrics();
128 return this.metrics;
129 }
130
131
132
133
134
135
136
137 boolean getKeepProgress() {
138 return keepProgress;
139 }
140
141 void setKeepProgress(boolean keepProgress) {
142 this.keepProgress = keepProgress;
143 }
144
145
146
147
148 void incrementBatchProgress(int batch) {
149 int currentBatch = progress.getBatch();
150 progress.setBatch(currentBatch + batch);
151 }
152
153
154
155
156 void incrementSizeProgress(long size) {
157 long currentSize = progress.getSize();
158 progress.setSize(currentSize + size);
159 }
160
161
162
163
164 void updateTimeProgress() {
165 progress.setTime(System.currentTimeMillis());
166 }
167
168 int getBatchProgress() {
169 return progress.getBatch();
170 }
171
172 long getSizeProgress() {
173 return progress.getSize();
174 }
175
176 long getTimeProgress() {
177 return progress.getTime();
178 }
179
180 void setProgress(int batchProgress, long sizeProgress, long timeProgress) {
181 setBatchProgress(batchProgress);
182 setSizeProgress(sizeProgress);
183 setTimeProgress(timeProgress);
184 }
185
186 void setSizeProgress(long sizeProgress) {
187 progress.setSize(sizeProgress);
188 }
189
190 void setBatchProgress(int batchProgress) {
191 progress.setBatch(batchProgress);
192 }
193
194 void setTimeProgress(long timeProgress) {
195 progress.setTime(timeProgress);
196 }
197
198
199
200
201
202 void clearProgress() {
203 progress.setFields(0, LimitFields.DEFAULT_SCOPE, 0, LimitFields.DEFAULT_SCOPE, 0);
204 }
205
206
207
208
209
210
211
212
213
214 NextState setScannerState(NextState state) {
215 if (!NextState.isValidState(state)) {
216 throw new IllegalArgumentException("Cannot set to invalid state: " + state);
217 }
218
219 this.scannerState = state;
220 return state;
221 }
222
223
224
225
226
227 boolean partialResultFormed() {
228 return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW
229 || scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW;
230 }
231
232
233
234
235
236 boolean hasBatchLimit(LimitScope checkerScope) {
237 return limits.canEnforceBatchLimitFromScope(checkerScope) && limits.getBatch() > 0;
238 }
239
240
241
242
243
244 boolean hasSizeLimit(LimitScope checkerScope) {
245 return limits.canEnforceSizeLimitFromScope(checkerScope) && limits.getSize() > 0;
246 }
247
248
249
250
251
252 boolean hasTimeLimit(LimitScope checkerScope) {
253 return limits.canEnforceTimeLimitFromScope(checkerScope) && limits.getTime() > 0;
254 }
255
256
257
258
259
260 boolean hasAnyLimit(LimitScope checkerScope) {
261 return hasBatchLimit(checkerScope) || hasSizeLimit(checkerScope) || hasTimeLimit(checkerScope);
262 }
263
264
265
266
267 void setSizeLimitScope(LimitScope scope) {
268 limits.setSizeScope(scope);
269 }
270
271
272
273
274 void setTimeLimitScope(LimitScope scope) {
275 limits.setTimeScope(scope);
276 }
277
278 int getBatchLimit() {
279 return limits.getBatch();
280 }
281
282 long getSizeLimit() {
283 return limits.getSize();
284 }
285
286 long getTimeLimit() {
287 return limits.getTime();
288 }
289
290
291
292
293
294 boolean checkBatchLimit(LimitScope checkerScope) {
295 return hasBatchLimit(checkerScope) && progress.getBatch() >= limits.getBatch();
296 }
297
298
299
300
301
302 boolean checkSizeLimit(LimitScope checkerScope) {
303 return hasSizeLimit(checkerScope) && progress.getSize() >= limits.getSize();
304 }
305
306
307
308
309
310
311 boolean checkTimeLimit(LimitScope checkerScope) {
312 return hasTimeLimit(checkerScope) && progress.getTime() >= limits.getTime();
313 }
314
315
316
317
318
319 boolean checkAnyLimitReached(LimitScope checkerScope) {
320 return checkSizeLimit(checkerScope) || checkBatchLimit(checkerScope)
321 || checkTimeLimit(checkerScope);
322 }
323
324 @Override
325 public String toString() {
326 StringBuilder sb = new StringBuilder();
327 sb.append("{");
328
329 sb.append("limits:");
330 sb.append(limits);
331
332 sb.append(", progress:");
333 sb.append(progress);
334
335 sb.append(", keepProgress:");
336 sb.append(keepProgress);
337
338 sb.append(", state:");
339 sb.append(scannerState);
340
341 sb.append("}");
342 return sb.toString();
343 }
344
345 public static Builder newBuilder() {
346 return new Builder();
347 }
348
349 public static Builder newBuilder(boolean keepProgress) {
350 return new Builder(keepProgress);
351 }
352
353 public static final class Builder {
354 boolean keepProgress = DEFAULT_KEEP_PROGRESS;
355 boolean trackMetrics = false;
356 LimitFields limits = new LimitFields();
357
358 private Builder() {
359 }
360
361 private Builder(boolean keepProgress) {
362 this.keepProgress = keepProgress;
363 }
364
365 public Builder setKeepProgress(boolean keepProgress) {
366 this.keepProgress = keepProgress;
367 return this;
368 }
369
370 public Builder setTrackMetrics(boolean trackMetrics) {
371 this.trackMetrics = trackMetrics;
372 return this;
373 }
374
375 public Builder setSizeLimit(LimitScope sizeScope, long sizeLimit) {
376 limits.setSize(sizeLimit);
377 limits.setSizeScope(sizeScope);
378 return this;
379 }
380
381 public Builder setTimeLimit(LimitScope timeScope, long timeLimit) {
382 limits.setTime(timeLimit);
383 limits.setTimeScope(timeScope);
384 return this;
385 }
386
387 public Builder setBatchLimit(int batchLimit) {
388 limits.setBatch(batchLimit);
389 return this;
390 }
391
392 public ScannerContext build() {
393 return new ScannerContext(keepProgress, limits, trackMetrics);
394 }
395 }
396
397
398
399
400 public enum NextState {
401 MORE_VALUES(true, false),
402 NO_MORE_VALUES(false, false),
403 SIZE_LIMIT_REACHED(true, true),
404
405
406
407
408
409 SIZE_LIMIT_REACHED_MID_ROW(true, true),
410 TIME_LIMIT_REACHED(true, true),
411
412
413
414
415
416 TIME_LIMIT_REACHED_MID_ROW(true, true),
417 BATCH_LIMIT_REACHED(true, true);
418
419 private boolean moreValues;
420 private boolean limitReached;
421
422 private NextState(boolean moreValues, boolean limitReached) {
423 this.moreValues = moreValues;
424 this.limitReached = limitReached;
425 }
426
427
428
429
430
431 public boolean hasMoreValues() {
432 return this.moreValues;
433 }
434
435
436
437
438 public boolean limitReached() {
439 return this.limitReached;
440 }
441
442 public static boolean isValidState(NextState state) {
443 return state != null;
444 }
445
446 public static boolean hasMoreValues(NextState state) {
447 return isValidState(state) && state.hasMoreValues();
448 }
449 }
450
451
452
453
454
455 public enum LimitScope {
456
457
458
459
460 BETWEEN_ROWS(0),
461
462
463
464
465
466 BETWEEN_CELLS(1);
467
468
469
470
471
472
473
474
475
476 int depth;
477
478 LimitScope(int depth) {
479 this.depth = depth;
480 }
481
482 int depth() {
483 return depth;
484 }
485
486
487
488
489
490
491
492 boolean canEnforceLimitFromScope(LimitScope checkerScope) {
493 return checkerScope != null && checkerScope.depth() <= depth;
494 }
495 }
496
497
498
499
500
501 private static class LimitFields {
502
503
504
505
506 private static int DEFAULT_BATCH = -1;
507 private static long DEFAULT_SIZE = -1L;
508 private static long DEFAULT_TIME = -1L;
509
510
511
512
513 private static final LimitScope DEFAULT_SCOPE = LimitScope.BETWEEN_ROWS;
514
515
516
517 int batch = DEFAULT_BATCH;
518
519 LimitScope sizeScope = DEFAULT_SCOPE;
520 long size = DEFAULT_SIZE;
521
522 LimitScope timeScope = DEFAULT_SCOPE;
523 long time = DEFAULT_TIME;
524
525
526
527
528 LimitFields() {
529 }
530
531 LimitFields(int batch, LimitScope sizeScope, long size, LimitScope timeScope, long time) {
532 setFields(batch, sizeScope, size, timeScope, time);
533 }
534
535 void copy(LimitFields limitsToCopy) {
536 if (limitsToCopy != null) {
537 setFields(limitsToCopy.getBatch(), limitsToCopy.getSizeScope(), limitsToCopy.getSize(),
538 limitsToCopy.getTimeScope(), limitsToCopy.getTime());
539 }
540 }
541
542
543
544
545
546
547
548 void setFields(int batch, LimitScope sizeScope, long size, LimitScope timeScope, long time) {
549 setBatch(batch);
550 setSizeScope(sizeScope);
551 setSize(size);
552 setTimeScope(timeScope);
553 setTime(time);
554 }
555
556 int getBatch() {
557 return this.batch;
558 }
559
560 void setBatch(int batch) {
561 this.batch = batch;
562 }
563
564
565
566
567
568 boolean canEnforceBatchLimitFromScope(LimitScope checkerScope) {
569 return LimitScope.BETWEEN_CELLS.canEnforceLimitFromScope(checkerScope);
570 }
571
572 long getSize() {
573 return this.size;
574 }
575
576 void setSize(long size) {
577 this.size = size;
578 }
579
580
581
582
583 LimitScope getSizeScope() {
584 return this.sizeScope;
585 }
586
587
588
589
590 void setSizeScope(LimitScope scope) {
591 this.sizeScope = scope;
592 }
593
594
595
596
597
598 boolean canEnforceSizeLimitFromScope(LimitScope checkerScope) {
599 return this.sizeScope.canEnforceLimitFromScope(checkerScope);
600 }
601
602 long getTime() {
603 return this.time;
604 }
605
606 void setTime(long time) {
607 this.time = time;
608 }
609
610
611
612
613 LimitScope getTimeScope() {
614 return this.timeScope;
615 }
616
617
618
619
620 void setTimeScope(LimitScope scope) {
621 this.timeScope = scope;
622 }
623
624
625
626
627
628 boolean canEnforceTimeLimitFromScope(LimitScope checkerScope) {
629 return this.sizeScope.canEnforceLimitFromScope(checkerScope);
630 }
631
632 @Override
633 public String toString() {
634 StringBuilder sb = new StringBuilder();
635 sb.append("{");
636
637 sb.append("batch:");
638 sb.append(batch);
639
640 sb.append(", size:");
641 sb.append(size);
642
643 sb.append(", sizeScope:");
644 sb.append(sizeScope);
645
646 sb.append(", time:");
647 sb.append(time);
648
649 sb.append(", timeScope:");
650 sb.append(timeScope);
651
652 sb.append("}");
653 return sb.toString();
654 }
655 }
656 }