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.client;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.HashMap;
023import java.util.List;
024import java.util.Map;
025import java.util.NavigableSet;
026import java.util.TreeMap;
027import java.util.TreeSet;
028import java.util.stream.Collectors;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
031import org.apache.hadoop.hbase.filter.Filter;
032import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
033import org.apache.hadoop.hbase.io.TimeRange;
034import org.apache.hadoop.hbase.security.access.Permission;
035import org.apache.hadoop.hbase.security.visibility.Authorizations;
036import org.apache.hadoop.hbase.util.Bytes;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041/**
042 * Used to perform Scan operations.
043 * <p>
044 * All operations are identical to {@link Get} with the exception of instantiation. Rather than
045 * specifying a single row, an optional startRow and stopRow may be defined. If rows are not
046 * specified, the Scanner will iterate over all rows.
047 * <p>
048 * To get all columns from all rows of a Table, create an instance with no constraints; use the
049 * {@link #Scan()} constructor. To constrain the scan to specific column families, call
050 * {@link #addFamily(byte[]) addFamily} for each family to retrieve on your Scan instance.
051 * <p>
052 * To get specific columns, call {@link #addColumn(byte[], byte[]) addColumn} for each column to
053 * retrieve.
054 * <p>
055 * To only retrieve columns within a specific range of version timestamps, call
056 * {@link #setTimeRange(long, long) setTimeRange}.
057 * <p>
058 * To only retrieve columns with a specific timestamp, call {@link #setTimestamp(long) setTimestamp}
059 * .
060 * <p>
061 * To limit the number of versions of each column to be returned, call {@link #readVersions(int)}.
062 * <p>
063 * To limit the maximum number of values returned for each call to next(), call
064 * {@link #setBatch(int) setBatch}.
065 * <p>
066 * To add a filter, call {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
067 * <p>
068 * For small scan, it is deprecated in 2.0.0. Now we have a {@link #setLimit(int)} method in Scan
069 * object which is used to tell RS how many rows we want. If the rows return reaches the limit, the
070 * RS will close the RegionScanner automatically. And we will also fetch data when openScanner in
071 * the new implementation, this means we can also finish a scan operation in one rpc call. And we
072 * have also introduced a {@link #setReadType(ReadType)} method. You can use this method to tell RS
073 * to use pread explicitly.
074 * <p>
075 * Expert: To explicitly disable server-side block caching for this scan, execute
076 * {@link #setCacheBlocks(boolean)}.
077 * <p>
078 * <em>Note:</em> Usage alters Scan instances. Internally, attributes are updated as the Scan runs
079 * and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when you go to
080 * clone a Scan instance or if you go to reuse a created Scan instance; safer is create a Scan
081 * instance per usage.
082 */
083@InterfaceAudience.Public
084public class Scan extends Query {
085  private static final Logger LOG = LoggerFactory.getLogger(Scan.class);
086
087  private static final String RAW_ATTR = "_raw_";
088
089  private byte[] startRow = HConstants.EMPTY_START_ROW;
090  private boolean includeStartRow = true;
091  private byte[] stopRow = HConstants.EMPTY_END_ROW;
092  private boolean includeStopRow = false;
093  private int maxVersions = 1;
094  private int batch = -1;
095
096  /**
097   * Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}.
098   * The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the
099   * cells in the row exceeded max result size on the server. Typically partial results will be
100   * combined client side into complete results before being delivered to the caller. However, if
101   * this flag is set, the caller is indicating that they do not mind seeing partial results (i.e.
102   * they understand that the results returned from the Scanner may only represent part of a
103   * particular row). In such a case, any attempt to combine the partials into a complete result on
104   * the client side will be skipped, and the caller will be able to see the exact results returned
105   * from the server.
106   */
107  private boolean allowPartialResults = false;
108
109  private int storeLimit = -1;
110  private int storeOffset = 0;
111
112  private static final String SCAN_ATTRIBUTES_METRICS_ENABLE = "scan.attributes.metrics.enable";
113
114  // If an application wants to use multiple scans over different tables each scan must
115  // define this attribute with the appropriate table name by calling
116  // scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(tableName))
117  static public final String SCAN_ATTRIBUTES_TABLE_NAME = "scan.attributes.table.name";
118
119  /**
120   * -1 means no caching specified and the value of {@link HConstants#HBASE_CLIENT_SCANNER_CACHING}
121   * (default to {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_CACHING}) will be used
122   */
123  private int caching = -1;
124  private long maxResultSize = -1;
125  private boolean cacheBlocks = true;
126  private boolean reversed = false;
127  private TimeRange tr = TimeRange.allTime();
128  private Map<byte[], NavigableSet<byte[]>> familyMap =
129    new TreeMap<byte[], NavigableSet<byte[]>>(Bytes.BYTES_COMPARATOR);
130  private Boolean asyncPrefetch = null;
131
132  /**
133   * Parameter name for client scanner sync/async prefetch toggle. When using async scanner,
134   * prefetching data from the server is done at the background. The parameter currently won't have
135   * any effect in the case that the user has set Scan#setSmall or Scan#setReversed
136   */
137  public static final String HBASE_CLIENT_SCANNER_ASYNC_PREFETCH =
138    "hbase.client.scanner.async.prefetch";
139
140  /**
141   * Default value of {@link #HBASE_CLIENT_SCANNER_ASYNC_PREFETCH}.
142   */
143  public static final boolean DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH = false;
144
145  /**
146   * The mvcc read point to use when open a scanner. Remember to clear it after switching regions as
147   * the mvcc is only valid within region scope.
148   */
149  private long mvccReadPoint = -1L;
150
151  /**
152   * The number of rows we want for this scan. We will terminate the scan if the number of return
153   * rows reaches this value.
154   */
155  private int limit = -1;
156
157  /**
158   * Control whether to use pread at server side.
159   */
160  private ReadType readType = ReadType.DEFAULT;
161
162  private boolean needCursorResult = false;
163
164  /**
165   * Create a Scan operation across all rows.
166   */
167  public Scan() {
168  }
169
170  /**
171   * Creates a new instance of this class while copying all values.
172   * @param scan The scan instance to copy from.
173   * @throws IOException When copying the values fails.
174   */
175  public Scan(Scan scan) throws IOException {
176    startRow = scan.getStartRow();
177    includeStartRow = scan.includeStartRow();
178    stopRow = scan.getStopRow();
179    includeStopRow = scan.includeStopRow();
180    maxVersions = scan.getMaxVersions();
181    batch = scan.getBatch();
182    storeLimit = scan.getMaxResultsPerColumnFamily();
183    storeOffset = scan.getRowOffsetPerColumnFamily();
184    caching = scan.getCaching();
185    maxResultSize = scan.getMaxResultSize();
186    cacheBlocks = scan.getCacheBlocks();
187    filter = scan.getFilter(); // clone?
188    loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
189    consistency = scan.getConsistency();
190    this.setIsolationLevel(scan.getIsolationLevel());
191    reversed = scan.isReversed();
192    asyncPrefetch = scan.isAsyncPrefetch();
193    allowPartialResults = scan.getAllowPartialResults();
194    tr = scan.getTimeRange(); // TimeRange is immutable
195    Map<byte[], NavigableSet<byte[]>> fams = scan.getFamilyMap();
196    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : fams.entrySet()) {
197      byte[] fam = entry.getKey();
198      NavigableSet<byte[]> cols = entry.getValue();
199      if (cols != null && cols.size() > 0) {
200        for (byte[] col : cols) {
201          addColumn(fam, col);
202        }
203      } else {
204        addFamily(fam);
205      }
206    }
207    for (Map.Entry<String, byte[]> attr : scan.getAttributesMap().entrySet()) {
208      setAttribute(attr.getKey(), attr.getValue());
209    }
210    for (Map.Entry<byte[], TimeRange> entry : scan.getColumnFamilyTimeRange().entrySet()) {
211      TimeRange tr = entry.getValue();
212      setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
213    }
214    this.mvccReadPoint = scan.getMvccReadPoint();
215    this.limit = scan.getLimit();
216    this.needCursorResult = scan.isNeedCursorResult();
217    setPriority(scan.getPriority());
218    readType = scan.getReadType();
219    super.setReplicaId(scan.getReplicaId());
220  }
221
222  /**
223   * Builds a scan object with the same specs as get.
224   * @param get get to model scan after
225   */
226  public Scan(Get get) {
227    this.startRow = get.getRow();
228    this.includeStartRow = true;
229    this.stopRow = get.getRow();
230    this.includeStopRow = true;
231    this.filter = get.getFilter();
232    this.cacheBlocks = get.getCacheBlocks();
233    this.maxVersions = get.getMaxVersions();
234    this.storeLimit = get.getMaxResultsPerColumnFamily();
235    this.storeOffset = get.getRowOffsetPerColumnFamily();
236    this.tr = get.getTimeRange();
237    this.familyMap = get.getFamilyMap();
238    this.asyncPrefetch = false;
239    this.consistency = get.getConsistency();
240    this.setIsolationLevel(get.getIsolationLevel());
241    this.loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue();
242    for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
243      setAttribute(attr.getKey(), attr.getValue());
244    }
245    for (Map.Entry<byte[], TimeRange> entry : get.getColumnFamilyTimeRange().entrySet()) {
246      TimeRange tr = entry.getValue();
247      setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
248    }
249    this.mvccReadPoint = -1L;
250    setPriority(get.getPriority());
251    super.setReplicaId(get.getReplicaId());
252  }
253
254  public boolean isGetScan() {
255    return includeStartRow && includeStopRow
256      && ClientUtil.areScanStartRowAndStopRowEqual(this.startRow, this.stopRow);
257  }
258
259  /**
260   * Get all columns from the specified family.
261   * <p>
262   * Overrides previous calls to addColumn for this family.
263   * @param family family name
264   */
265  public Scan addFamily(byte[] family) {
266    familyMap.remove(family);
267    familyMap.put(family, null);
268    return this;
269  }
270
271  /**
272   * Get the column from the specified family with the specified qualifier.
273   * <p>
274   * Overrides previous calls to addFamily for this family.
275   * @param family    family name
276   * @param qualifier column qualifier
277   */
278  public Scan addColumn(byte[] family, byte[] qualifier) {
279    NavigableSet<byte[]> set = familyMap.get(family);
280    if (set == null) {
281      set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
282      familyMap.put(family, set);
283    }
284    if (qualifier == null) {
285      qualifier = HConstants.EMPTY_BYTE_ARRAY;
286    }
287    set.add(qualifier);
288    return this;
289  }
290
291  /**
292   * Get versions of columns only within the specified timestamp range, [minStamp, maxStamp). Note,
293   * default maximum versions to return is 1. If your time range spans more than one version and you
294   * want all versions returned, up the number of versions beyond the default.
295   * @param minStamp minimum timestamp value, inclusive
296   * @param maxStamp maximum timestamp value, exclusive
297   * @see #readAllVersions()
298   * @see #readVersions(int)
299   */
300  public Scan setTimeRange(long minStamp, long maxStamp) throws IOException {
301    tr = TimeRange.between(minStamp, maxStamp);
302    return this;
303  }
304
305  /**
306   * Get versions of columns with the specified timestamp. Note, default maximum versions to return
307   * is 1. If your time range spans more than one version and you want all versions returned, up the
308   * number of versions beyond the defaut.
309   * @param timestamp version timestamp
310   * @see #readAllVersions()
311   * @see #readVersions(int)
312   */
313  public Scan setTimestamp(long timestamp) {
314    try {
315      tr = TimeRange.at(timestamp);
316    } catch (Exception e) {
317      // This should never happen, unless integer overflow or something extremely wrong...
318      LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
319      throw e;
320    }
321
322    return this;
323  }
324
325  @Override
326  public Scan setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {
327    return (Scan) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
328  }
329
330  /**
331   * Set the start row of the scan.
332   * <p>
333   * If the specified row does not exist, the Scanner will start from the next closest row after the
334   * specified row.
335   * <p>
336   * <b>Note:</b> <strong>Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or
337   * {@link #setStartStopRowForPrefixScan(byte[])}.</strong> Doing so will make the scan result
338   * unexpected or even undefined.
339   * </p>
340   * @param startRow row to start scanner at or after
341   * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
342   *                                  exceeds {@link HConstants#MAX_ROW_LENGTH})
343   */
344  public Scan withStartRow(byte[] startRow) {
345    return withStartRow(startRow, true);
346  }
347
348  /**
349   * Set the start row of the scan.
350   * <p>
351   * If the specified row does not exist, or the {@code inclusive} is {@code false}, the Scanner
352   * will start from the next closest row after the specified row.
353   * <p>
354   * <b>Note:</b> <strong>Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or
355   * {@link #setStartStopRowForPrefixScan(byte[])}.</strong> Doing so will make the scan result
356   * unexpected or even undefined.
357   * </p>
358   * @param startRow  row to start scanner at or after
359   * @param inclusive whether we should include the start row when scan
360   * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
361   *                                  exceeds {@link HConstants#MAX_ROW_LENGTH})
362   */
363  public Scan withStartRow(byte[] startRow, boolean inclusive) {
364    if (Bytes.len(startRow) > HConstants.MAX_ROW_LENGTH) {
365      throw new IllegalArgumentException("startRow's length must be less than or equal to "
366        + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
367    }
368    this.startRow = startRow;
369    this.includeStartRow = inclusive;
370    return this;
371  }
372
373  /**
374   * Set the stop row of the scan.
375   * <p>
376   * The scan will include rows that are lexicographically less than the provided stopRow.
377   * <p>
378   * <b>Note:</b> <strong>Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or
379   * {@link #setStartStopRowForPrefixScan(byte[])}.</strong> Doing so will make the scan result
380   * unexpected or even undefined.
381   * </p>
382   * @param stopRow row to end at (exclusive)
383   * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
384   *                                  exceeds {@link HConstants#MAX_ROW_LENGTH})
385   */
386  public Scan withStopRow(byte[] stopRow) {
387    return withStopRow(stopRow, false);
388  }
389
390  /**
391   * Set the stop row of the scan.
392   * <p>
393   * The scan will include rows that are lexicographically less than (or equal to if
394   * {@code inclusive} is {@code true}) the provided stopRow.
395   * <p>
396   * <b>Note:</b> <strong>Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or
397   * {@link #setStartStopRowForPrefixScan(byte[])}.</strong> Doing so will make the scan result
398   * unexpected or even undefined.
399   * </p>
400   * @param stopRow   row to end at
401   * @param inclusive whether we should include the stop row when scan
402   * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
403   *                                  exceeds {@link HConstants#MAX_ROW_LENGTH})
404   */
405  public Scan withStopRow(byte[] stopRow, boolean inclusive) {
406    if (Bytes.len(stopRow) > HConstants.MAX_ROW_LENGTH) {
407      throw new IllegalArgumentException("stopRow's length must be less than or equal to "
408        + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
409    }
410    this.stopRow = stopRow;
411    this.includeStopRow = inclusive;
412    return this;
413  }
414
415  /**
416   * <p>
417   * Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey
418   * starts with the specified prefix.
419   * </p>
420   * <p>
421   * This is a utility method that converts the desired rowPrefix into the appropriate values for
422   * the startRow and stopRow to achieve the desired result.
423   * </p>
424   * <p>
425   * This can safely be used in combination with setFilter.
426   * </p>
427   * <p>
428   * <strong>This CANNOT be used in combination with withStartRow and/or withStopRow.</strong> Such
429   * a combination will yield unexpected and even undefined results.
430   * </p>
431   * @param rowPrefix the prefix all rows must start with. (Set <i>null</i> to remove the filter.)
432   * @deprecated since 2.5.0, will be removed in 4.0.0. The name of this method is considered to be
433   *             confusing as it does not use a {@link Filter} but uses setting the startRow and
434   *             stopRow instead. Use {@link #setStartStopRowForPrefixScan(byte[])} instead.
435   */
436  @Deprecated
437  public Scan setRowPrefixFilter(byte[] rowPrefix) {
438    return setStartStopRowForPrefixScan(rowPrefix);
439  }
440
441  /**
442   * <p>
443   * Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey
444   * starts with the specified prefix.
445   * </p>
446   * <p>
447   * This is a utility method that converts the desired rowPrefix into the appropriate values for
448   * the startRow and stopRow to achieve the desired result.
449   * </p>
450   * <p>
451   * This can safely be used in combination with setFilter.
452   * </p>
453   * <p>
454   * <strong>This CANNOT be used in combination with withStartRow and/or withStopRow.</strong> Such
455   * a combination will yield unexpected and even undefined results.
456   * </p>
457   * @param rowPrefix the prefix all rows must start with. (Set <i>null</i> to remove the filter.)
458   */
459  public Scan setStartStopRowForPrefixScan(byte[] rowPrefix) {
460    if (rowPrefix == null) {
461      withStartRow(HConstants.EMPTY_START_ROW);
462      withStopRow(HConstants.EMPTY_END_ROW);
463    } else {
464      this.withStartRow(rowPrefix);
465      this.withStopRow(ClientUtil.calculateTheClosestNextRowKeyForPrefix(rowPrefix));
466    }
467    return this;
468  }
469
470  /**
471   * Get all available versions.
472   */
473  public Scan readAllVersions() {
474    this.maxVersions = Integer.MAX_VALUE;
475    return this;
476  }
477
478  /**
479   * Get up to the specified number of versions of each column.
480   * @param versions specified number of versions for each column
481   */
482  public Scan readVersions(int versions) {
483    this.maxVersions = versions;
484    return this;
485  }
486
487  /**
488   * Set the maximum number of cells to return for each call to next(). Callers should be aware that
489   * this is not equivalent to calling {@link #setAllowPartialResults(boolean)}. If you don't allow
490   * partial results, the number of cells in each Result must equal to your batch setting unless it
491   * is the last Result for current row. So this method is helpful in paging queries. If you just
492   * want to prevent OOM at client, use setAllowPartialResults(true) is better.
493   * @param batch the maximum number of values
494   * @see Result#mayHaveMoreCellsInRow()
495   */
496  public Scan setBatch(int batch) {
497    if (this.hasFilter() && this.filter.hasFilterRow()) {
498      throw new IncompatibleFilterException(
499        "Cannot set batch on a scan using a filter" + " that returns true for filter.hasFilterRow");
500    }
501    this.batch = batch;
502    return this;
503  }
504
505  /**
506   * Set the maximum number of values to return per row per Column Family
507   * @param limit the maximum number of values returned / row / CF
508   */
509  public Scan setMaxResultsPerColumnFamily(int limit) {
510    this.storeLimit = limit;
511    return this;
512  }
513
514  /**
515   * Set offset for the row per Column Family.
516   * @param offset is the number of kvs that will be skipped.
517   */
518  public Scan setRowOffsetPerColumnFamily(int offset) {
519    this.storeOffset = offset;
520    return this;
521  }
522
523  /**
524   * Set the number of rows for caching that will be passed to scanners. If not set, the
525   * Configuration setting {@link HConstants#HBASE_CLIENT_SCANNER_CACHING} will apply. Higher
526   * caching values will enable faster scanners but will use more memory.
527   * @param caching the number of rows for caching
528   */
529  public Scan setCaching(int caching) {
530    this.caching = caching;
531    return this;
532  }
533
534  /** Returns the maximum result size in bytes. See {@link #setMaxResultSize(long)} */
535  public long getMaxResultSize() {
536    return maxResultSize;
537  }
538
539  /**
540   * Set the maximum result size. The default is -1; this means that no specific maximum result size
541   * will be set for this scan, and the global configured value will be used instead. (Defaults to
542   * unlimited).
543   * @param maxResultSize The maximum result size in bytes.
544   */
545  public Scan setMaxResultSize(long maxResultSize) {
546    this.maxResultSize = maxResultSize;
547    return this;
548  }
549
550  @Override
551  public Scan setFilter(Filter filter) {
552    super.setFilter(filter);
553    return this;
554  }
555
556  /**
557   * Setting the familyMap
558   * @param familyMap map of family to qualifier
559   */
560  public Scan setFamilyMap(Map<byte[], NavigableSet<byte[]>> familyMap) {
561    this.familyMap = familyMap;
562    return this;
563  }
564
565  /**
566   * Getting the familyMap
567   */
568  public Map<byte[], NavigableSet<byte[]>> getFamilyMap() {
569    return this.familyMap;
570  }
571
572  /** Returns the number of families in familyMap */
573  public int numFamilies() {
574    if (hasFamilies()) {
575      return this.familyMap.size();
576    }
577    return 0;
578  }
579
580  /** Returns true if familyMap is non empty, false otherwise */
581  public boolean hasFamilies() {
582    return !this.familyMap.isEmpty();
583  }
584
585  /** Returns the keys of the familyMap */
586  public byte[][] getFamilies() {
587    if (hasFamilies()) {
588      return this.familyMap.keySet().toArray(new byte[0][0]);
589    }
590    return null;
591  }
592
593  /** Returns the startrow */
594  public byte[] getStartRow() {
595    return this.startRow;
596  }
597
598  /** Returns if we should include start row when scan */
599  public boolean includeStartRow() {
600    return includeStartRow;
601  }
602
603  /** Returns the stoprow */
604  public byte[] getStopRow() {
605    return this.stopRow;
606  }
607
608  /** Returns if we should include stop row when scan */
609  public boolean includeStopRow() {
610    return includeStopRow;
611  }
612
613  /** Returns the max number of versions to fetch */
614  public int getMaxVersions() {
615    return this.maxVersions;
616  }
617
618  /** Returns maximum number of values to return for a single call to next() */
619  public int getBatch() {
620    return this.batch;
621  }
622
623  /** Returns maximum number of values to return per row per CF */
624  public int getMaxResultsPerColumnFamily() {
625    return this.storeLimit;
626  }
627
628  /**
629   * Method for retrieving the scan's offset per row per column family (#kvs to be skipped)
630   * @return row offset
631   */
632  public int getRowOffsetPerColumnFamily() {
633    return this.storeOffset;
634  }
635
636  /** Returns caching the number of rows fetched when calling next on a scanner */
637  public int getCaching() {
638    return this.caching;
639  }
640
641  /** Returns TimeRange */
642  public TimeRange getTimeRange() {
643    return this.tr;
644  }
645
646  /** Returns RowFilter */
647  @Override
648  public Filter getFilter() {
649    return filter;
650  }
651
652  /** Returns true is a filter has been specified, false if not */
653  public boolean hasFilter() {
654    return filter != null;
655  }
656
657  /**
658   * Set whether blocks should be cached for this Scan.
659   * <p>
660   * This is true by default. When true, default settings of the table and family are used (this
661   * will never override caching blocks if the block cache is disabled for that family or entirely).
662   * @param cacheBlocks if false, default settings are overridden and blocks will not be cached
663   */
664  public Scan setCacheBlocks(boolean cacheBlocks) {
665    this.cacheBlocks = cacheBlocks;
666    return this;
667  }
668
669  /**
670   * Get whether blocks should be cached for this Scan.
671   * @return true if default caching should be used, false if blocks should not be cached
672   */
673  public boolean getCacheBlocks() {
674    return cacheBlocks;
675  }
676
677  /**
678   * Set whether this scan is a reversed one
679   * <p>
680   * This is false by default which means forward(normal) scan.
681   * @param reversed if true, scan will be backward order
682   */
683  public Scan setReversed(boolean reversed) {
684    this.reversed = reversed;
685    return this;
686  }
687
688  /**
689   * Get whether this scan is a reversed one.
690   * @return true if backward scan, false if forward(default) scan
691   */
692  public boolean isReversed() {
693    return reversed;
694  }
695
696  /**
697   * Setting whether the caller wants to see the partial results when server returns
698   * less-than-expected cells. It is helpful while scanning a huge row to prevent OOM at client. By
699   * default this value is false and the complete results will be assembled client side before being
700   * delivered to the caller.
701   * @see Result#mayHaveMoreCellsInRow()
702   * @see #setBatch(int)
703   */
704  public Scan setAllowPartialResults(final boolean allowPartialResults) {
705    this.allowPartialResults = allowPartialResults;
706    return this;
707  }
708
709  /**
710   * Returns true when the constructor of this scan understands that the results they will see may
711   * only represent a partial portion of a row. The entire row would be retrieved by subsequent
712   * calls to {@link ResultScanner#next()}
713   */
714  public boolean getAllowPartialResults() {
715    return allowPartialResults;
716  }
717
718  @Override
719  public Scan setLoadColumnFamiliesOnDemand(boolean value) {
720    return (Scan) super.setLoadColumnFamiliesOnDemand(value);
721  }
722
723  /**
724   * Compile the table and column family (i.e. schema) information into a String. Useful for parsing
725   * and aggregation by debugging, logging, and administration tools.
726   */
727  @Override
728  public Map<String, Object> getFingerprint() {
729    Map<String, Object> map = new HashMap<>();
730    List<String> families = new ArrayList<>();
731    if (this.familyMap.isEmpty()) {
732      map.put("families", "ALL");
733      return map;
734    } else {
735      map.put("families", families);
736    }
737    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
738      families.add(Bytes.toStringBinary(entry.getKey()));
739    }
740    return map;
741  }
742
743  /**
744   * Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a
745   * Map along with the fingerprinted information. Useful for debugging, logging, and administration
746   * tools.
747   * @param maxCols a limit on the number of columns output prior to truncation
748   */
749  @Override
750  public Map<String, Object> toMap(int maxCols) {
751    // start with the fingerprint map and build on top of it
752    Map<String, Object> map = getFingerprint();
753    // map from families to column list replaces fingerprint's list of families
754    Map<String, List<String>> familyColumns = new HashMap<>();
755    map.put("families", familyColumns);
756    // add scalar information first
757    map.put("startRow", Bytes.toStringBinary(this.startRow));
758    map.put("stopRow", Bytes.toStringBinary(this.stopRow));
759    map.put("maxVersions", this.maxVersions);
760    map.put("batch", this.batch);
761    map.put("caching", this.caching);
762    map.put("maxResultSize", this.maxResultSize);
763    map.put("cacheBlocks", this.cacheBlocks);
764    map.put("loadColumnFamiliesOnDemand", this.loadColumnFamiliesOnDemand);
765    List<Long> timeRange = new ArrayList<>(2);
766    timeRange.add(this.tr.getMin());
767    timeRange.add(this.tr.getMax());
768    map.put("timeRange", timeRange);
769    int colCount = 0;
770    // iterate through affected families and list out up to maxCols columns
771    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
772      List<String> columns = new ArrayList<>();
773      familyColumns.put(Bytes.toStringBinary(entry.getKey()), columns);
774      if (entry.getValue() == null) {
775        colCount++;
776        --maxCols;
777        columns.add("ALL");
778      } else {
779        colCount += entry.getValue().size();
780        if (maxCols <= 0) {
781          continue;
782        }
783        for (byte[] column : entry.getValue()) {
784          if (--maxCols <= 0) {
785            continue;
786          }
787          columns.add(Bytes.toStringBinary(column));
788        }
789      }
790    }
791    map.put("totalColumns", colCount);
792    if (this.filter != null) {
793      map.put("filter", this.filter.toString());
794    }
795    // add the id if set
796    if (getId() != null) {
797      map.put("id", getId());
798    }
799    map.put("includeStartRow", includeStartRow);
800    map.put("includeStopRow", includeStopRow);
801    map.put("allowPartialResults", allowPartialResults);
802    map.put("storeLimit", storeLimit);
803    map.put("storeOffset", storeOffset);
804    map.put("reversed", reversed);
805    if (null != asyncPrefetch) {
806      map.put("asyncPrefetch", asyncPrefetch);
807    }
808    map.put("mvccReadPoint", mvccReadPoint);
809    map.put("limit", limit);
810    map.put("readType", readType);
811    map.put("needCursorResult", needCursorResult);
812    map.put("targetReplicaId", targetReplicaId);
813    map.put("consistency", consistency);
814    if (!colFamTimeRangeMap.isEmpty()) {
815      Map<String, List<Long>> colFamTimeRangeMapStr = colFamTimeRangeMap.entrySet().stream()
816        .collect(Collectors.toMap((e) -> Bytes.toStringBinary(e.getKey()), e -> {
817          TimeRange value = e.getValue();
818          List<Long> rangeList = new ArrayList<>();
819          rangeList.add(value.getMin());
820          rangeList.add(value.getMax());
821          return rangeList;
822        }));
823
824      map.put("colFamTimeRangeMap", colFamTimeRangeMapStr);
825    }
826    map.put("priority", getPriority());
827    return map;
828  }
829
830  /**
831   * Enable/disable "raw" mode for this scan. If "raw" is enabled the scan will return all delete
832   * marker and deleted rows that have not been collected, yet. This is mostly useful for Scan on
833   * column families that have KEEP_DELETED_ROWS enabled. It is an error to specify any column when
834   * "raw" is set.
835   * @param raw True/False to enable/disable "raw" mode.
836   */
837  public Scan setRaw(boolean raw) {
838    setAttribute(RAW_ATTR, Bytes.toBytes(raw));
839    return this;
840  }
841
842  /** Returns True if this Scan is in "raw" mode. */
843  public boolean isRaw() {
844    byte[] attr = getAttribute(RAW_ATTR);
845    return attr == null ? false : Bytes.toBoolean(attr);
846  }
847
848  @Override
849  public Scan setAttribute(String name, byte[] value) {
850    return (Scan) super.setAttribute(name, value);
851  }
852
853  @Override
854  public Scan setId(String id) {
855    return (Scan) super.setId(id);
856  }
857
858  @Override
859  public Scan setAuthorizations(Authorizations authorizations) {
860    return (Scan) super.setAuthorizations(authorizations);
861  }
862
863  @Override
864  public Scan setACL(Map<String, Permission> perms) {
865    return (Scan) super.setACL(perms);
866  }
867
868  @Override
869  public Scan setACL(String user, Permission perms) {
870    return (Scan) super.setACL(user, perms);
871  }
872
873  @Override
874  public Scan setConsistency(Consistency consistency) {
875    return (Scan) super.setConsistency(consistency);
876  }
877
878  @Override
879  public Scan setReplicaId(int Id) {
880    return (Scan) super.setReplicaId(Id);
881  }
882
883  @Override
884  public Scan setIsolationLevel(IsolationLevel level) {
885    return (Scan) super.setIsolationLevel(level);
886  }
887
888  @Override
889  public Scan setPriority(int priority) {
890    return (Scan) super.setPriority(priority);
891  }
892
893  /**
894   * Enable collection of {@link ScanMetrics}. For advanced users.
895   * @param enabled Set to true to enable accumulating scan metrics
896   */
897  public Scan setScanMetricsEnabled(final boolean enabled) {
898    setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.valueOf(enabled)));
899    return this;
900  }
901
902  /** Returns True if collection of scan metrics is enabled. For advanced users. */
903  public boolean isScanMetricsEnabled() {
904    byte[] attr = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
905    return attr == null ? false : Bytes.toBoolean(attr);
906  }
907
908  public Boolean isAsyncPrefetch() {
909    return asyncPrefetch;
910  }
911
912  /**
913   * @deprecated Since 3.0.0, will be removed in 4.0.0. After building sync client upon async
914   *             client, the implementation is always 'async prefetch', so this flag is useless now.
915   */
916  @Deprecated
917  public Scan setAsyncPrefetch(boolean asyncPrefetch) {
918    this.asyncPrefetch = asyncPrefetch;
919    return this;
920  }
921
922  /** Returns the limit of rows for this scan */
923  public int getLimit() {
924    return limit;
925  }
926
927  /**
928   * Set the limit of rows for this scan. We will terminate the scan if the number of returned rows
929   * reaches this value.
930   * <p>
931   * This condition will be tested at last, after all other conditions such as stopRow, filter, etc.
932   * @param limit the limit of rows for this scan
933   */
934  public Scan setLimit(int limit) {
935    this.limit = limit;
936    return this;
937  }
938
939  /**
940   * Call this when you only want to get one row. It will set {@code limit} to {@code 1}, and also
941   * set {@code readType} to {@link ReadType#PREAD}.
942   */
943  public Scan setOneRowLimit() {
944    return setLimit(1).setReadType(ReadType.PREAD);
945  }
946
947  @InterfaceAudience.Public
948  public enum ReadType {
949    DEFAULT,
950    STREAM,
951    PREAD
952  }
953
954  /** Returns the read type for this scan */
955  public ReadType getReadType() {
956    return readType;
957  }
958
959  /**
960   * Set the read type for this scan.
961   * <p>
962   * Notice that we may choose to use pread even if you specific {@link ReadType#STREAM} here. For
963   * example, we will always use pread if this is a get scan.
964   */
965  public Scan setReadType(ReadType readType) {
966    this.readType = readType;
967    return this;
968  }
969
970  /**
971   * Get the mvcc read point used to open a scanner.
972   */
973  long getMvccReadPoint() {
974    return mvccReadPoint;
975  }
976
977  /**
978   * Set the mvcc read point used to open a scanner.
979   */
980  Scan setMvccReadPoint(long mvccReadPoint) {
981    this.mvccReadPoint = mvccReadPoint;
982    return this;
983  }
984
985  /**
986   * Set the mvcc read point to -1 which means do not use it.
987   */
988  Scan resetMvccReadPoint() {
989    return setMvccReadPoint(-1L);
990  }
991
992  /**
993   * When the server is slow or we scan a table with many deleted data or we use a sparse filter,
994   * the server will response heartbeat to prevent timeout. However the scanner will return a Result
995   * only when client can do it. So if there are many heartbeats, the blocking time on
996   * ResultScanner#next() may be very long, which is not friendly to online services. Set this to
997   * true then you can get a special Result whose #isCursor() returns true and is not contains any
998   * real data. It only tells you where the server has scanned. You can call next to continue
999   * scanning or open a new scanner with this row key as start row whenever you want. Users can get
1000   * a cursor when and only when there is a response from the server but we can not return a Result
1001   * to users, for example, this response is a heartbeat or there are partial cells but users do not
1002   * allow partial result. Now the cursor is in row level which means the special Result will only
1003   * contains a row key. {@link Result#isCursor()} {@link Result#getCursor()} {@link Cursor}
1004   */
1005  public Scan setNeedCursorResult(boolean needCursorResult) {
1006    this.needCursorResult = needCursorResult;
1007    return this;
1008  }
1009
1010  public boolean isNeedCursorResult() {
1011    return needCursorResult;
1012  }
1013
1014  /**
1015   * Create a new Scan with a cursor. It only set the position information like start row key. The
1016   * others (like cfs, stop row, limit) should still be filled in by the user.
1017   * {@link Result#isCursor()} {@link Result#getCursor()} {@link Cursor}
1018   */
1019  public static Scan createScanFromCursor(Cursor cursor) {
1020    return new Scan().withStartRow(cursor.getRow());
1021  }
1022}