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 boolean midRowResultFormed() {
236 return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW
237 || scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW
238 || scannerState == NextState.BATCH_LIMIT_REACHED;
239 }
240
241
242
243
244
245 boolean hasBatchLimit(LimitScope checkerScope) {
246 return limits.canEnforceBatchLimitFromScope(checkerScope) && limits.getBatch() > 0;
247 }
248
249
250
251
252
253 boolean hasSizeLimit(LimitScope checkerScope) {
254 return limits.canEnforceSizeLimitFromScope(checkerScope) && limits.getSize() > 0;
255 }
256
257
258
259
260
261 boolean hasTimeLimit(LimitScope checkerScope) {
262 return limits.canEnforceTimeLimitFromScope(checkerScope) && limits.getTime() > 0;
263 }
264
265
266
267
268
269 boolean hasAnyLimit(LimitScope checkerScope) {
270 return hasBatchLimit(checkerScope) || hasSizeLimit(checkerScope) || hasTimeLimit(checkerScope);
271 }
272
273
274
275
276 void setSizeLimitScope(LimitScope scope) {
277 limits.setSizeScope(scope);
278 }
279
280
281
282
283 void setTimeLimitScope(LimitScope scope) {
284 limits.setTimeScope(scope);
285 }
286
287 int getBatchLimit() {
288 return limits.getBatch();
289 }
290
291 long getSizeLimit() {
292 return limits.getSize();
293 }
294
295 long getTimeLimit() {
296 return limits.getTime();
297 }
298
299
300
301
302
303 boolean checkBatchLimit(LimitScope checkerScope) {
304 return hasBatchLimit(checkerScope) && progress.getBatch() >= limits.getBatch();
305 }
306
307
308
309
310
311 boolean checkSizeLimit(LimitScope checkerScope) {
312 return hasSizeLimit(checkerScope) && progress.getSize() >= limits.getSize();
313 }
314
315
316
317
318
319
320 boolean checkTimeLimit(LimitScope checkerScope) {
321 return hasTimeLimit(checkerScope) && progress.getTime() >= limits.getTime();
322 }
323
324
325
326
327
328 boolean checkAnyLimitReached(LimitScope checkerScope) {
329 return checkSizeLimit(checkerScope) || checkBatchLimit(checkerScope)
330 || checkTimeLimit(checkerScope);
331 }
332
333 @Override
334 public String toString() {
335 StringBuilder sb = new StringBuilder();
336 sb.append("{");
337
338 sb.append("limits:");
339 sb.append(limits);
340
341 sb.append(", progress:");
342 sb.append(progress);
343
344 sb.append(", keepProgress:");
345 sb.append(keepProgress);
346
347 sb.append(", state:");
348 sb.append(scannerState);
349
350 sb.append("}");
351 return sb.toString();
352 }
353
354 public static Builder newBuilder() {
355 return new Builder();
356 }
357
358 public static Builder newBuilder(boolean keepProgress) {
359 return new Builder(keepProgress);
360 }
361
362 public static final class Builder {
363 boolean keepProgress = DEFAULT_KEEP_PROGRESS;
364 boolean trackMetrics = false;
365 LimitFields limits = new LimitFields();
366
367 private Builder() {
368 }
369
370 private Builder(boolean keepProgress) {
371 this.keepProgress = keepProgress;
372 }
373
374 public Builder setKeepProgress(boolean keepProgress) {
375 this.keepProgress = keepProgress;
376 return this;
377 }
378
379 public Builder setTrackMetrics(boolean trackMetrics) {
380 this.trackMetrics = trackMetrics;
381 return this;
382 }
383
384 public Builder setSizeLimit(LimitScope sizeScope, long sizeLimit) {
385 limits.setSize(sizeLimit);
386 limits.setSizeScope(sizeScope);
387 return this;
388 }
389
390 public Builder setTimeLimit(LimitScope timeScope, long timeLimit) {
391 limits.setTime(timeLimit);
392 limits.setTimeScope(timeScope);
393 return this;
394 }
395
396 public Builder setBatchLimit(int batchLimit) {
397 limits.setBatch(batchLimit);
398 return this;
399 }
400
401 public ScannerContext build() {
402 return new ScannerContext(keepProgress, limits, trackMetrics);
403 }
404 }
405
406
407
408
409 public enum NextState {
410 MORE_VALUES(true, false),
411 NO_MORE_VALUES(false, false),
412 SIZE_LIMIT_REACHED(true, true),
413
414
415
416
417
418 SIZE_LIMIT_REACHED_MID_ROW(true, true),
419 TIME_LIMIT_REACHED(true, true),
420
421
422
423
424
425 TIME_LIMIT_REACHED_MID_ROW(true, true),
426 BATCH_LIMIT_REACHED(true, true);
427
428 private boolean moreValues;
429 private boolean limitReached;
430
431 private NextState(boolean moreValues, boolean limitReached) {
432 this.moreValues = moreValues;
433 this.limitReached = limitReached;
434 }
435
436
437
438
439
440 public boolean hasMoreValues() {
441 return this.moreValues;
442 }
443
444
445
446
447 public boolean limitReached() {
448 return this.limitReached;
449 }
450
451 public static boolean isValidState(NextState state) {
452 return state != null;
453 }
454
455 public static boolean hasMoreValues(NextState state) {
456 return isValidState(state) && state.hasMoreValues();
457 }
458 }
459
460
461
462
463
464 public enum LimitScope {
465
466
467
468
469 BETWEEN_ROWS(0),
470
471
472
473
474
475 BETWEEN_CELLS(1);
476
477
478
479
480
481
482
483
484
485 int depth;
486
487 LimitScope(int depth) {
488 this.depth = depth;
489 }
490
491 int depth() {
492 return depth;
493 }
494
495
496
497
498
499
500
501 boolean canEnforceLimitFromScope(LimitScope checkerScope) {
502 return checkerScope != null && checkerScope.depth() <= depth;
503 }
504 }
505
506
507
508
509
510 private static class LimitFields {
511
512
513
514
515 private static int DEFAULT_BATCH = -1;
516 private static long DEFAULT_SIZE = -1L;
517 private static long DEFAULT_TIME = -1L;
518
519
520
521
522 private static final LimitScope DEFAULT_SCOPE = LimitScope.BETWEEN_ROWS;
523
524
525
526 int batch = DEFAULT_BATCH;
527
528 LimitScope sizeScope = DEFAULT_SCOPE;
529 long size = DEFAULT_SIZE;
530
531 LimitScope timeScope = DEFAULT_SCOPE;
532 long time = DEFAULT_TIME;
533
534
535
536
537 LimitFields() {
538 }
539
540 LimitFields(int batch, LimitScope sizeScope, long size, LimitScope timeScope, long time) {
541 setFields(batch, sizeScope, size, timeScope, time);
542 }
543
544 void copy(LimitFields limitsToCopy) {
545 if (limitsToCopy != null) {
546 setFields(limitsToCopy.getBatch(), limitsToCopy.getSizeScope(), limitsToCopy.getSize(),
547 limitsToCopy.getTimeScope(), limitsToCopy.getTime());
548 }
549 }
550
551
552
553
554
555
556
557 void setFields(int batch, LimitScope sizeScope, long size, LimitScope timeScope, long time) {
558 setBatch(batch);
559 setSizeScope(sizeScope);
560 setSize(size);
561 setTimeScope(timeScope);
562 setTime(time);
563 }
564
565 int getBatch() {
566 return this.batch;
567 }
568
569 void setBatch(int batch) {
570 this.batch = batch;
571 }
572
573
574
575
576
577 boolean canEnforceBatchLimitFromScope(LimitScope checkerScope) {
578 return LimitScope.BETWEEN_CELLS.canEnforceLimitFromScope(checkerScope);
579 }
580
581 long getSize() {
582 return this.size;
583 }
584
585 void setSize(long size) {
586 this.size = size;
587 }
588
589
590
591
592 LimitScope getSizeScope() {
593 return this.sizeScope;
594 }
595
596
597
598
599 void setSizeScope(LimitScope scope) {
600 this.sizeScope = scope;
601 }
602
603
604
605
606
607 boolean canEnforceSizeLimitFromScope(LimitScope checkerScope) {
608 return this.sizeScope.canEnforceLimitFromScope(checkerScope);
609 }
610
611 long getTime() {
612 return this.time;
613 }
614
615 void setTime(long time) {
616 this.time = time;
617 }
618
619
620
621
622 LimitScope getTimeScope() {
623 return this.timeScope;
624 }
625
626
627
628
629 void setTimeScope(LimitScope scope) {
630 this.timeScope = scope;
631 }
632
633
634
635
636
637 boolean canEnforceTimeLimitFromScope(LimitScope checkerScope) {
638 return this.timeScope.canEnforceLimitFromScope(checkerScope);
639 }
640
641 @Override
642 public String toString() {
643 StringBuilder sb = new StringBuilder();
644 sb.append("{");
645
646 sb.append("batch:");
647 sb.append(batch);
648
649 sb.append(", size:");
650 sb.append(size);
651
652 sb.append(", sizeScope:");
653 sb.append(sizeScope);
654
655 sb.append(", time:");
656 sb.append(time);
657
658 sb.append(", timeScope:");
659 sb.append(timeScope);
660
661 sb.append("}");
662 return sb.toString();
663 }
664 }
665 }