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