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