001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import java.util.List;
021import org.apache.hadoop.hbase.Cell;
022import org.apache.hadoop.hbase.HBaseInterfaceAudience;
023import org.apache.hadoop.hbase.client.metrics.ServerSideScanMetrics;
024import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
025import org.apache.yetus.audience.InterfaceAudience;
026import org.apache.yetus.audience.InterfaceStability;
027
028/**
029 * ScannerContext instances encapsulate limit tracking AND progress towards those limits during
030 * invocations of {@link InternalScanner#next(java.util.List)} and
031 * {@link RegionScanner#next(java.util.List)}.
032 * <p>
033 * A ScannerContext instance should be updated periodically throughout execution whenever progress
034 * towards a limit has been made. Each limit can be checked via the appropriate checkLimit method.
035 * <p>
036 * Once a limit has been reached, the scan will stop. The invoker of
037 * {@link InternalScanner#next(java.util.List)} or {@link RegionScanner#next(java.util.List)} can
038 * use the appropriate check*Limit methods to see exactly which limits have been reached.
039 * Alternatively, {@link #checkAnyLimitReached(LimitScope)} is provided to see if ANY limit was
040 * reached
041 * <p>
042 * {@link NoLimitScannerContext#NO_LIMIT} is an immutable static definition that can be used
043 * whenever a {@link ScannerContext} is needed but limits do not need to be enforced.
044 * <p>
045 * NOTE: It is important that this class only ever expose setter methods that can be safely skipped
046 * when limits should be NOT enforced. This is because of the necessary immutability of the class
047 * {@link NoLimitScannerContext}. If a setter cannot be safely skipped, the immutable nature of
048 * {@link NoLimitScannerContext} will lead to incorrect behavior.
049 */
050@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
051@InterfaceStability.Evolving
052public class ScannerContext {
053
054  LimitFields limits;
055  /**
056   * A different set of progress fields. Only include batch, dataSize and heapSize. Compare to
057   * LimitFields, ProgressFields doesn't contain time field. As we save a deadline in LimitFields,
058   * so use {@link EnvironmentEdgeManager#currentTime()} directly when check time limit.
059   */
060  ProgressFields progress;
061
062  /**
063   * The state of the scanner after the invocation of {@link InternalScanner#next(java.util.List)}
064   * or {@link RegionScanner#next(java.util.List)}.
065   */
066  NextState scannerState;
067  private static final NextState DEFAULT_STATE = NextState.MORE_VALUES;
068
069  /**
070   * Used as an indication to invocations of {@link InternalScanner#next(java.util.List)} and
071   * {@link RegionScanner#next(java.util.List)} that, if true, the progress tracked within this
072   * {@link ScannerContext} instance should be considered while evaluating the limits. Useful for
073   * enforcing a set of limits across multiple calls (i.e. the limit may not be reached in a single
074   * invocation, but any progress made should be considered in future invocations)
075   * <p>
076   * Defaulting this value to false means that, by default, any tracked progress will be wiped clean
077   * on invocations to {@link InternalScanner#next(java.util.List)} and
078   * {@link RegionScanner#next(java.util.List)} and the call will be treated as though no progress
079   * has been made towards the limits so far.
080   * <p>
081   * This is an important mechanism. Users of Internal/Region scanners expect that they can define
082   * some limits and then repeatedly invoke {@link InternalScanner#next(List)} or
083   * {@link RegionScanner#next(List)} where each invocation respects these limits separately.
084   * <p>
085   * For example:
086   *
087   * <pre>
088   *  {@code
089   * ScannerContext context = new ScannerContext.newBuilder().setBatchLimit(5).build();
090   * RegionScanner scanner = ...
091   * List<Cell> results = new ArrayList<Cell>();
092   * while(scanner.next(results, context)) {
093   *   // Do something with a batch of 5 cells
094   * }
095   * }
096   * </pre>
097   *
098   * However, in the case of RPCs, the server wants to be able to define a set of limits for a
099   * particular RPC request and have those limits respected across multiple invocations. This means
100   * that the progress made towards the limits in earlier calls will be saved and considered in
101   * future invocations
102   */
103  boolean keepProgress;
104  private static boolean DEFAULT_KEEP_PROGRESS = false;
105
106  /**
107   * Allows temporarily ignoring limits and skipping tracking of batch and size progress. Used when
108   * skipping to the next row, in which case all processed cells are thrown away so should not count
109   * towards progress.
110   */
111  boolean skippingRow = false;
112
113  private Cell lastPeekedCell = null;
114
115  // Set this to true will have the same behavior with reaching the time limit.
116  // This is used when you want to make the current RSRpcService.scan returns immediately. For
117  // example, when we want to switch from pread to stream, we can only do it after the rpc call is
118  // returned.
119  private boolean returnImmediately;
120
121  /**
122   * Tracks the relevant server side metrics during scans. null when metrics should not be tracked
123   */
124  final ServerSideScanMetrics metrics;
125
126  ScannerContext(boolean keepProgress, LimitFields limitsToCopy, boolean trackMetrics) {
127    this.limits = new LimitFields();
128    if (limitsToCopy != null) {
129      this.limits.copy(limitsToCopy);
130    }
131
132    // Progress fields are initialized to 0
133    progress = new ProgressFields(0, 0, 0, 0);
134
135    this.keepProgress = keepProgress;
136    this.scannerState = DEFAULT_STATE;
137    this.metrics = trackMetrics ? new ServerSideScanMetrics() : null;
138  }
139
140  public boolean isTrackingMetrics() {
141    return this.metrics != null;
142  }
143
144  /**
145   * Get the metrics instance. Should only be called after a call to {@link #isTrackingMetrics()}
146   * has been made to confirm that metrics are indeed being tracked.
147   * @return {@link ServerSideScanMetrics} instance that is tracking metrics for this scan
148   */
149  public ServerSideScanMetrics getMetrics() {
150    assert isTrackingMetrics();
151    return this.metrics;
152  }
153
154  /**
155   * @return true if the progress tracked so far in this instance will be considered during an
156   *         invocation of {@link InternalScanner#next(java.util.List)} or
157   *         {@link RegionScanner#next(java.util.List)}. false when the progress tracked so far
158   *         should not be considered and should instead be wiped away via {@link #clearProgress()}.
159   *         This only applies to per-row progress, like batch and data/heap size. Block size is
160   *         never reset because it tracks all of the blocks scanned for an entire request.
161   */
162  boolean getKeepProgress() {
163    return keepProgress;
164  }
165
166  void setKeepProgress(boolean keepProgress) {
167    this.keepProgress = keepProgress;
168  }
169
170  /**
171   * In this mode, only block size progress is tracked, and limits are ignored. We set this mode
172   * when skipping to next row, in which case all cells returned a thrown away so should not count
173   * towards progress.
174   * @return true if we are in skipping row mode.
175   */
176  public boolean getSkippingRow() {
177    return skippingRow;
178  }
179
180  /**
181   * @param skippingRow set true to cause disabling of collecting per-cell progress or enforcing any
182   *                    limits. This is used when trying to skip over all cells in a row, in which
183   *                    case those cells are thrown away so should not count towards progress.
184   */
185  void setSkippingRow(boolean skippingRow) {
186    this.skippingRow = skippingRow;
187  }
188
189  /**
190   * Progress towards the batch limit has been made. Increment internal tracking of batch progress
191   */
192  void incrementBatchProgress(int batch) {
193    if (skippingRow) {
194      return;
195    }
196    int currentBatch = progress.getBatch();
197    progress.setBatch(currentBatch + batch);
198  }
199
200  /**
201   * Progress towards the size limit has been made. Increment internal tracking of size progress
202   */
203  void incrementSizeProgress(long dataSize, long heapSize) {
204    if (skippingRow) {
205      return;
206    }
207    long curDataSize = progress.getDataSize();
208    progress.setDataSize(curDataSize + dataSize);
209    long curHeapSize = progress.getHeapSize();
210    progress.setHeapSize(curHeapSize + heapSize);
211  }
212
213  /**
214   * Update the time progress.
215   * @deprecated will be removed in 3.0
216   */
217  @Deprecated
218  void updateTimeProgress() {
219
220  }
221
222  /**
223   * Progress towards the block limit has been made. Increment internal track of block progress
224   */
225  void incrementBlockProgress(int blockSize) {
226    if (blockSize > 0) {
227      long curBlockSize = progress.getBlockSize();
228      progress.setBlockSize(curBlockSize + blockSize);
229    }
230  }
231
232  int getBatchProgress() {
233    return progress.getBatch();
234  }
235
236  long getDataSizeProgress() {
237    return progress.getDataSize();
238  }
239
240  long getHeapSizeProgress() {
241    return progress.getHeapSize();
242  }
243
244  /**
245   * @deprecated will be removed in 3.0
246   */
247  @Deprecated
248  long getTimeProgress() {
249    return EnvironmentEdgeManager.currentTime();
250  }
251
252  /**
253   * @deprecated will be removed in 3.0
254   */
255  @Deprecated
256  void setProgress(int batchProgress, long sizeProgress, long heapSizeProgress, long timeProgress) {
257    setProgress(batchProgress, sizeProgress, heapSizeProgress);
258  }
259
260  long getBlockSizeProgress() {
261    return progress.getBlockSize();
262  }
263
264  void setProgress(int batchProgress, long sizeProgress, long heapSizeProgress) {
265    setBatchProgress(batchProgress);
266    setSizeProgress(sizeProgress, heapSizeProgress);
267  }
268
269  void setSizeProgress(long dataSizeProgress, long heapSizeProgress) {
270    progress.setDataSize(dataSizeProgress);
271    progress.setHeapSize(heapSizeProgress);
272  }
273
274  void setBatchProgress(int batchProgress) {
275    progress.setBatch(batchProgress);
276  }
277
278  /**
279   * @deprecated will be removed in 3.0
280   */
281  @Deprecated
282  void setTimeProgress(long timeProgress) {
283  }
284
285  /**
286   * Clear away any progress that has been made so far. All progress fields are reset to initial
287   * values. Only clears progress that should reset between rows. {@link #getBlockSizeProgress()} is
288   * not reset because it increments for all blocks scanned whether the result is included or
289   * filtered.
290   */
291  void clearProgress() {
292    progress.setFields(0, 0, 0, getBlockSizeProgress());
293  }
294
295  /**
296   * Note that this is not a typical setter. This setter returns the {@link NextState} that was
297   * passed in so that methods can be invoked against the new state. Furthermore, this pattern
298   * allows the {@link NoLimitScannerContext} to cleanly override this setter and simply return the
299   * new state, thus preserving the immutability of {@link NoLimitScannerContext}
300   * @return The state that was passed in.
301   */
302  NextState setScannerState(NextState state) {
303    if (!NextState.isValidState(state)) {
304      throw new IllegalArgumentException("Cannot set to invalid state: " + state);
305    }
306
307    this.scannerState = state;
308    return state;
309  }
310
311  /**
312   * @return true when we have more cells for the current row. This usually because we have reached
313   *         a limit in the middle of a row
314   */
315  boolean mayHaveMoreCellsInRow() {
316    return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW
317      || scannerState == NextState.TIME_LIMIT_REACHED_MID_ROW
318      || scannerState == NextState.BATCH_LIMIT_REACHED;
319  }
320
321  /** Returns true if the batch limit can be enforced in the checker's scope */
322  boolean hasBatchLimit(LimitScope checkerScope) {
323    return limits.canEnforceBatchLimitFromScope(checkerScope) && limits.getBatch() > 0;
324  }
325
326  /** Returns true if the size limit can be enforced in the checker's scope */
327  boolean hasSizeLimit(LimitScope checkerScope) {
328    return limits.canEnforceSizeLimitFromScope(checkerScope)
329      && (limits.getDataSize() > 0 || limits.getHeapSize() > 0 || limits.getBlockSize() > 0);
330  }
331
332  /** Returns true if the time limit can be enforced in the checker's scope */
333  boolean hasTimeLimit(LimitScope checkerScope) {
334    return limits.canEnforceTimeLimitFromScope(checkerScope)
335      && (limits.getTime() > 0 || returnImmediately);
336  }
337
338  /** Returns true if any limit can be enforced within the checker's scope */
339  boolean hasAnyLimit(LimitScope checkerScope) {
340    return hasBatchLimit(checkerScope) || hasSizeLimit(checkerScope) || hasTimeLimit(checkerScope);
341  }
342
343  /**
344   * @param scope The scope in which the size limit will be enforced
345   */
346  void setSizeLimitScope(LimitScope scope) {
347    limits.setSizeScope(scope);
348  }
349
350  /**
351   * @param scope The scope in which the time limit will be enforced
352   */
353  void setTimeLimitScope(LimitScope scope) {
354    limits.setTimeScope(scope);
355  }
356
357  int getBatchLimit() {
358    return limits.getBatch();
359  }
360
361  long getDataSizeLimit() {
362    return limits.getDataSize();
363  }
364
365  long getTimeLimit() {
366    return limits.getTime();
367  }
368
369  /**
370   * @param checkerScope The scope that the limit is being checked from
371   * @return true when the limit is enforceable from the checker's scope and it has been reached
372   */
373  boolean checkBatchLimit(LimitScope checkerScope) {
374    return !skippingRow && hasBatchLimit(checkerScope) && progress.getBatch() >= limits.getBatch();
375  }
376
377  /**
378   * @param checkerScope The scope that the limit is being checked from
379   * @return true when the limit is enforceable from the checker's scope and it has been reached
380   */
381  boolean checkSizeLimit(LimitScope checkerScope) {
382    return !skippingRow && hasSizeLimit(checkerScope)
383      && (progress.getDataSize() >= limits.getDataSize()
384        || progress.getHeapSize() >= limits.getHeapSize()
385        || progress.getBlockSize() >= limits.getBlockSize());
386  }
387
388  /**
389   * @param checkerScope The scope that the limit is being checked from. The time limit is always
390   *                     checked against {@link EnvironmentEdgeManager.currentTime}
391   * @return true when the limit is enforceable from the checker's scope and it has been reached
392   */
393  boolean checkTimeLimit(LimitScope checkerScope) {
394    return !skippingRow && hasTimeLimit(checkerScope)
395      && (returnImmediately || EnvironmentEdgeManager.currentTime() >= limits.getTime());
396  }
397
398  /**
399   * @param checkerScope The scope that the limits are being checked from
400   * @return true when some limit is enforceable from the checker's scope and it has been reached
401   */
402  boolean checkAnyLimitReached(LimitScope checkerScope) {
403    return checkSizeLimit(checkerScope) || checkBatchLimit(checkerScope)
404      || checkTimeLimit(checkerScope);
405  }
406
407  Cell getLastPeekedCell() {
408    return lastPeekedCell;
409  }
410
411  void setLastPeekedCell(Cell lastPeekedCell) {
412    this.lastPeekedCell = lastPeekedCell;
413  }
414
415  void returnImmediately() {
416    this.returnImmediately = true;
417  }
418
419  @Override
420  public String toString() {
421    StringBuilder sb = new StringBuilder();
422    sb.append("{");
423
424    sb.append("limits:");
425    sb.append(limits);
426
427    sb.append(", progress:");
428    sb.append(progress);
429
430    sb.append(", keepProgress:");
431    sb.append(keepProgress);
432
433    sb.append(", state:");
434    sb.append(scannerState);
435
436    sb.append("}");
437    return sb.toString();
438  }
439
440  public static Builder newBuilder() {
441    return new Builder();
442  }
443
444  public static Builder newBuilder(boolean keepProgress) {
445    return new Builder(keepProgress);
446  }
447
448  public static final class Builder {
449    boolean keepProgress = DEFAULT_KEEP_PROGRESS;
450    boolean trackMetrics = false;
451    LimitFields limits = new LimitFields();
452
453    private Builder() {
454    }
455
456    private Builder(boolean keepProgress) {
457      this.keepProgress = keepProgress;
458    }
459
460    public Builder setKeepProgress(boolean keepProgress) {
461      this.keepProgress = keepProgress;
462      return this;
463    }
464
465    public Builder setTrackMetrics(boolean trackMetrics) {
466      this.trackMetrics = trackMetrics;
467      return this;
468    }
469
470    public Builder setSizeLimit(LimitScope sizeScope, long dataSizeLimit, long heapSizeLimit,
471      long blockSizeLimit) {
472      limits.setDataSize(dataSizeLimit);
473      limits.setHeapSize(heapSizeLimit);
474      limits.setSizeScope(sizeScope);
475      limits.setBlockSize(blockSizeLimit);
476      return this;
477    }
478
479    public Builder setTimeLimit(LimitScope timeScope, long timeLimit) {
480      limits.setTime(timeLimit);
481      limits.setTimeScope(timeScope);
482      return this;
483    }
484
485    public Builder setBatchLimit(int batchLimit) {
486      limits.setBatch(batchLimit);
487      return this;
488    }
489
490    public ScannerContext build() {
491      return new ScannerContext(keepProgress, limits, trackMetrics);
492    }
493  }
494
495  /**
496   * The possible states a scanner may be in following a call to {@link InternalScanner#next(List)}
497   */
498  public enum NextState {
499    MORE_VALUES(true, false),
500    NO_MORE_VALUES(false, false),
501    SIZE_LIMIT_REACHED(true, true),
502
503    /**
504     * Special case of size limit reached to indicate that the size limit was reached in the middle
505     * of a row and thus a partial results was formed
506     */
507    SIZE_LIMIT_REACHED_MID_ROW(true, true),
508    TIME_LIMIT_REACHED(true, true),
509
510    /**
511     * Special case of time limit reached to indicate that the time limit was reached in the middle
512     * of a row and thus a partial results was formed
513     */
514    TIME_LIMIT_REACHED_MID_ROW(true, true),
515    BATCH_LIMIT_REACHED(true, true);
516
517    private final boolean moreValues;
518    private final boolean limitReached;
519
520    private NextState(boolean moreValues, boolean limitReached) {
521      this.moreValues = moreValues;
522      this.limitReached = limitReached;
523    }
524
525    /**
526     * @return true when the state indicates that more values may follow those that have been
527     *         returned
528     */
529    public boolean hasMoreValues() {
530      return this.moreValues;
531    }
532
533    /** Returns true when the state indicates that a limit has been reached and scan should stop */
534    public boolean limitReached() {
535      return this.limitReached;
536    }
537
538    public static boolean isValidState(NextState state) {
539      return state != null;
540    }
541
542    public static boolean hasMoreValues(NextState state) {
543      return isValidState(state) && state.hasMoreValues();
544    }
545  }
546
547  /**
548   * The various scopes where a limit can be enforced. Used to differentiate when a limit should be
549   * enforced or not.
550   */
551  public enum LimitScope {
552    /**
553     * Enforcing a limit between rows means that the limit will not be considered until all the
554     * cells for a particular row have been retrieved
555     */
556    BETWEEN_ROWS(0),
557
558    /**
559     * Enforcing a limit between cells means that the limit will be considered after each full cell
560     * has been retrieved
561     */
562    BETWEEN_CELLS(1);
563
564    /**
565     * When enforcing a limit, we must check that the scope is appropriate for enforcement.
566     * <p>
567     * To communicate this concept, each scope has a depth. A limit will be enforced if the depth of
568     * the checker's scope is less than or equal to the limit's scope. This means that when checking
569     * limits, the checker must know their own scope (i.e. are they checking the limits between
570     * rows, between cells, etc...)
571     */
572    final int depth;
573
574    LimitScope(int depth) {
575      this.depth = depth;
576    }
577
578    final int depth() {
579      return depth;
580    }
581
582    /**
583     * @param checkerScope The scope in which the limit is being checked
584     * @return true when the checker is in a scope that indicates the limit can be enforced. Limits
585     *         can be enforced from "higher or equal" scopes (i.e. the checker's scope is at a
586     *         lesser depth than the limit)
587     */
588    boolean canEnforceLimitFromScope(LimitScope checkerScope) {
589      return checkerScope != null && checkerScope.depth() <= depth;
590    }
591  }
592
593  /**
594   * The different fields that can be used as limits in calls to
595   * {@link InternalScanner#next(java.util.List)} and {@link RegionScanner#next(java.util.List)}
596   */
597  private static class LimitFields {
598    /**
599     * Default values of the limit fields. Defined such that if a field does NOT change from its
600     * default, it will not be enforced
601     */
602    private static int DEFAULT_BATCH = -1;
603    private static long DEFAULT_SIZE = -1L;
604    private static long DEFAULT_TIME = -1L;
605
606    /**
607     * Default scope that is assigned to a limit if a scope is not specified.
608     */
609    private static final LimitScope DEFAULT_SCOPE = LimitScope.BETWEEN_ROWS;
610
611    // The batch limit will always be enforced between cells, thus, there isn't a field to hold the
612    // batch scope
613    int batch = DEFAULT_BATCH;
614
615    LimitScope sizeScope = DEFAULT_SCOPE;
616    // The sum of cell data sizes(key + value). The Cell data might be in on heap or off heap area.
617    long dataSize = DEFAULT_SIZE;
618    // The sum of heap space occupied by all tracked cells. This includes Cell POJO's overhead as
619    // such AND data cells of Cells which are in on heap area.
620    long heapSize = DEFAULT_SIZE;
621    // The total amount of block bytes that have been loaded in order to process cells for the
622    // request.
623    long blockSize = DEFAULT_SIZE;
624
625    LimitScope timeScope = DEFAULT_SCOPE;
626    long time = DEFAULT_TIME;
627
628    /**
629     * Fields keep their default values.
630     */
631    LimitFields() {
632    }
633
634    void copy(LimitFields limitsToCopy) {
635      if (limitsToCopy != null) {
636        setFields(limitsToCopy.getBatch(), limitsToCopy.getSizeScope(), limitsToCopy.getDataSize(),
637          limitsToCopy.getHeapSize(), limitsToCopy.getBlockSize(), limitsToCopy.getTimeScope(),
638          limitsToCopy.getTime());
639      }
640    }
641
642    /**
643     * Set all fields together.
644     */
645    void setFields(int batch, LimitScope sizeScope, long dataSize, long heapSize, long blockSize,
646      LimitScope timeScope, long time) {
647      setBatch(batch);
648      setSizeScope(sizeScope);
649      setDataSize(dataSize);
650      setHeapSize(heapSize);
651      setBlockSize(blockSize);
652      setTimeScope(timeScope);
653      setTime(time);
654    }
655
656    int getBatch() {
657      return this.batch;
658    }
659
660    void setBatch(int batch) {
661      this.batch = batch;
662    }
663
664    /** Returns true when the limit can be enforced from the scope of the checker */
665    boolean canEnforceBatchLimitFromScope(LimitScope checkerScope) {
666      return LimitScope.BETWEEN_CELLS.canEnforceLimitFromScope(checkerScope);
667    }
668
669    long getDataSize() {
670      return this.dataSize;
671    }
672
673    long getHeapSize() {
674      return this.heapSize;
675    }
676
677    long getBlockSize() {
678      return this.blockSize;
679    }
680
681    void setDataSize(long dataSize) {
682      this.dataSize = dataSize;
683    }
684
685    void setHeapSize(long heapSize) {
686      this.heapSize = heapSize;
687    }
688
689    void setBlockSize(long blockSize) {
690      this.blockSize = blockSize;
691    }
692
693    /** Returns {@link LimitScope} indicating scope in which the size limit is enforced */
694    LimitScope getSizeScope() {
695      return this.sizeScope;
696    }
697
698    /**
699     * Change the scope in which the size limit is enforced
700     */
701    void setSizeScope(LimitScope scope) {
702      this.sizeScope = scope;
703    }
704
705    /** Returns true when the limit can be enforced from the scope of the checker */
706    boolean canEnforceSizeLimitFromScope(LimitScope checkerScope) {
707      return this.sizeScope.canEnforceLimitFromScope(checkerScope);
708    }
709
710    long getTime() {
711      return this.time;
712    }
713
714    void setTime(long time) {
715      this.time = time;
716    }
717
718    /** Returns {@link LimitScope} indicating scope in which the time limit is enforced */
719    LimitScope getTimeScope() {
720      return this.timeScope;
721    }
722
723    /**
724     * Change the scope in which the time limit is enforced
725     */
726    void setTimeScope(LimitScope scope) {
727      this.timeScope = scope;
728    }
729
730    /** Returns true when the limit can be enforced from the scope of the checker */
731    boolean canEnforceTimeLimitFromScope(LimitScope checkerScope) {
732      return this.timeScope.canEnforceLimitFromScope(checkerScope);
733    }
734
735    @Override
736    public String toString() {
737      StringBuilder sb = new StringBuilder();
738      sb.append("{");
739
740      sb.append("batch:");
741      sb.append(batch);
742
743      sb.append(", dataSize:");
744      sb.append(dataSize);
745
746      sb.append(", heapSize:");
747      sb.append(heapSize);
748
749      sb.append(", blockSize:");
750      sb.append(blockSize);
751
752      sb.append(", sizeScope:");
753      sb.append(sizeScope);
754
755      sb.append(", time:");
756      sb.append(time);
757
758      sb.append(", timeScope:");
759      sb.append(timeScope);
760
761      sb.append("}");
762      return sb.toString();
763    }
764  }
765
766  private static class ProgressFields {
767
768    private static int DEFAULT_BATCH = -1;
769    private static long DEFAULT_SIZE = -1L;
770
771    // The batch limit will always be enforced between cells, thus, there isn't a field to hold the
772    // batch scope
773    int batch = DEFAULT_BATCH;
774
775    // The sum of cell data sizes(key + value). The Cell data might be in on heap or off heap area.
776    long dataSize = DEFAULT_SIZE;
777    // The sum of heap space occupied by all tracked cells. This includes Cell POJO's overhead as
778    // such AND data cells of Cells which are in on heap area.
779    long heapSize = DEFAULT_SIZE;
780    // The total amount of block bytes that have been loaded in order to process cells for the
781    // request.
782    long blockSize = DEFAULT_SIZE;
783
784    ProgressFields(int batch, long size, long heapSize, long blockSize) {
785      setFields(batch, size, heapSize, blockSize);
786    }
787
788    /**
789     * Set all fields together.
790     */
791    void setFields(int batch, long dataSize, long heapSize, long blockSize) {
792      setBatch(batch);
793      setDataSize(dataSize);
794      setHeapSize(heapSize);
795      setBlockSize(blockSize);
796    }
797
798    int getBatch() {
799      return this.batch;
800    }
801
802    void setBatch(int batch) {
803      this.batch = batch;
804    }
805
806    long getDataSize() {
807      return this.dataSize;
808    }
809
810    long getHeapSize() {
811      return this.heapSize;
812    }
813
814    long getBlockSize() {
815      return this.blockSize;
816    }
817
818    void setDataSize(long dataSize) {
819      this.dataSize = dataSize;
820    }
821
822    void setBlockSize(long blockSize) {
823      this.blockSize = blockSize;
824    }
825
826    void setHeapSize(long heapSize) {
827      this.heapSize = heapSize;
828    }
829
830    @Override
831    public String toString() {
832      StringBuilder sb = new StringBuilder();
833      sb.append("{");
834
835      sb.append("batch:");
836      sb.append(batch);
837
838      sb.append(", dataSize:");
839      sb.append(dataSize);
840
841      sb.append(", heapSize:");
842      sb.append(heapSize);
843
844      sb.append(", blockSize:");
845      sb.append(blockSize);
846
847      sb.append("}");
848      return sb.toString();
849    }
850  }
851}