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