View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Arrays;
25  import java.util.HashMap;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.NavigableSet;
29  import java.util.TreeMap;
30  import java.util.TreeSet;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.classification.InterfaceStability;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
38  import org.apache.hadoop.hbase.filter.Filter;
39  import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
40  import org.apache.hadoop.hbase.io.TimeRange;
41  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
42  import org.apache.hadoop.hbase.security.access.Permission;
43  import org.apache.hadoop.hbase.security.visibility.Authorizations;
44  import org.apache.hadoop.hbase.util.Bytes;
45  
46  /**
47   * Used to perform Scan operations.
48   * <p>
49   * All operations are identical to {@link Get} with the exception of
50   * instantiation.  Rather than specifying a single row, an optional startRow
51   * and stopRow may be defined.  If rows are not specified, the Scanner will
52   * iterate over all rows.
53   * <p>
54   * To scan everything for each row, instantiate a Scan object.
55   * <p>
56   * To modify scanner caching for just this scan, use {@link #setCaching(int) setCaching}.
57   * If caching is NOT set, we will use the caching value of the hosting {@link Table}.
58   * In addition to row caching, it is possible to specify a
59   * maximum result size, using {@link #setMaxResultSize(long)}. When both are used,
60   * single server requests are limited by either number of rows or maximum result size, whichever
61   * limit comes first.
62   * <p>
63   * To further define the scope of what to get when scanning, perform additional
64   * methods as outlined below.
65   * <p>
66   * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
67   * for each family to retrieve.
68   * <p>
69   * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn}
70   * for each column to retrieve.
71   * <p>
72   * To only retrieve columns within a specific range of version timestamps,
73   * execute {@link #setTimeRange(long, long) setTimeRange}.
74   * <p>
75   * To only retrieve columns with a specific timestamp, execute
76   * {@link #setTimeStamp(long) setTimestamp}.
77   * <p>
78   * To limit the number of versions of each column to be returned, execute
79   * {@link #setMaxVersions(int) setMaxVersions}.
80   * <p>
81   * To limit the maximum number of values returned for each call to next(),
82   * execute {@link #setBatch(int) setBatch}.
83   * <p>
84   * To add a filter, execute {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
85   * <p>
86   * Expert: To explicitly disable server-side block caching for this scan,
87   * execute {@link #setCacheBlocks(boolean)}.
88   * <p><em>Note:</em> Usage alters Scan instances. Internally, attributes are updated as the Scan
89   * runs and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when
90   * you go to clone a Scan instance or if you go to reuse a created Scan instance; safer is create
91   * a Scan instance per usage.
92   */
93  @InterfaceAudience.Public
94  @InterfaceStability.Stable
95  public class Scan extends Query {
96    private static final Log LOG = LogFactory.getLog(Scan.class);
97  
98    private static final String RAW_ATTR = "_raw_";
99  
100   private byte [] startRow = HConstants.EMPTY_START_ROW;
101   private byte [] stopRow  = HConstants.EMPTY_END_ROW;
102   private int maxVersions = 1;
103   private int batch = -1;
104 
105   /**
106    * Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}.
107    * The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the
108    * cells in the row exceeded max result size on the server. Typically partial results will be
109    * combined client side into complete results before being delivered to the caller. However, if
110    * this flag is set, the caller is indicating that they do not mind seeing partial results (i.e.
111    * they understand that the results returned from the Scanner may only represent part of a
112    * particular row). In such a case, any attempt to combine the partials into a complete result on
113    * the client side will be skipped, and the caller will be able to see the exact results returned
114    * from the server.
115    */
116   private boolean allowPartialResults = false;
117 
118   private int storeLimit = -1;
119   private int storeOffset = 0;
120   private boolean getScan;
121 
122   /**
123    * @deprecated since 1.0.0. Use {@link #setScanMetricsEnabled(boolean)}
124    */
125   // Make private or remove.
126   @Deprecated
127   static public final String SCAN_ATTRIBUTES_METRICS_ENABLE = "scan.attributes.metrics.enable";
128 
129   /**
130    * Use {@link #getScanMetrics()}
131    */
132   // Make this private or remove.
133   @Deprecated
134   static public final String SCAN_ATTRIBUTES_METRICS_DATA = "scan.attributes.metrics.data";
135 
136   // If an application wants to use multiple scans over different tables each scan must
137   // define this attribute with the appropriate table name by calling
138   // scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(tableName))
139   static public final String SCAN_ATTRIBUTES_TABLE_NAME = "scan.attributes.table.name";
140 
141   /**
142    * @deprecated without replacement
143    *             This is now a no-op, SEEKs and SKIPs are optimizated automatically.
144    *             Will be removed in 2.0+
145    */
146   @Deprecated
147   public static final String HINT_LOOKAHEAD = "_look_ahead_";
148 
149   /*
150    * -1 means no caching
151    */
152   private int caching = -1;
153   private long maxResultSize = -1;
154   private boolean cacheBlocks = true;
155   private boolean reversed = false;
156   private TimeRange tr = new TimeRange();
157   private Map<byte [], NavigableSet<byte []>> familyMap =
158     new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
159   private Boolean loadColumnFamiliesOnDemand = null;
160 
161   /**
162    * Set it true for small scan to get better performance
163    *
164    * Small scan should use pread and big scan can use seek + read
165    *
166    * seek + read is fast but can cause two problem (1) resource contention (2)
167    * cause too much network io
168    *
169    * [89-fb] Using pread for non-compaction read request
170    * https://issues.apache.org/jira/browse/HBASE-7266
171    *
172    * On the other hand, if setting it true, we would do
173    * openScanner,next,closeScanner in one RPC call. It means the better
174    * performance for small scan. [HBASE-9488].
175    *
176    * Generally, if the scan range is within one data block(64KB), it could be
177    * considered as a small scan.
178    */
179   private boolean small = false;
180 
181   /**
182    * Create a Scan operation across all rows.
183    */
184   public Scan() {}
185 
186   public Scan(byte [] startRow, Filter filter) {
187     this(startRow);
188     this.filter = filter;
189   }
190 
191   /**
192    * Create a Scan operation starting at the specified row.
193    * <p>
194    * If the specified row does not exist, the Scanner will start from the
195    * next closest row after the specified row.
196    * @param startRow row to start scanner at or after
197    */
198   public Scan(byte [] startRow) {
199     this.startRow = startRow;
200   }
201 
202   /**
203    * Create a Scan operation for the range of rows specified.
204    * @param startRow row to start scanner at or after (inclusive)
205    * @param stopRow row to stop scanner before (exclusive)
206    */
207   public Scan(byte [] startRow, byte [] stopRow) {
208     this.startRow = startRow;
209     this.stopRow = stopRow;
210     //if the startRow and stopRow both are empty, it is not a Get
211     this.getScan = isStartRowAndEqualsStopRow();
212   }
213 
214   /**
215    * Creates a new instance of this class while copying all values.
216    *
217    * @param scan  The scan instance to copy from.
218    * @throws IOException When copying the values fails.
219    */
220   public Scan(Scan scan) throws IOException {
221     startRow = scan.getStartRow();
222     stopRow  = scan.getStopRow();
223     maxVersions = scan.getMaxVersions();
224     batch = scan.getBatch();
225     storeLimit = scan.getMaxResultsPerColumnFamily();
226     storeOffset = scan.getRowOffsetPerColumnFamily();
227     caching = scan.getCaching();
228     maxResultSize = scan.getMaxResultSize();
229     cacheBlocks = scan.getCacheBlocks();
230     getScan = scan.isGetScan();
231     filter = scan.getFilter(); // clone?
232     loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
233     consistency = scan.getConsistency();
234     reversed = scan.isReversed();
235     small = scan.isSmall();
236     allowPartialResults = scan.getAllowPartialResults();
237     TimeRange ctr = scan.getTimeRange();
238     tr = new TimeRange(ctr.getMin(), ctr.getMax());
239     Map<byte[], NavigableSet<byte[]>> fams = scan.getFamilyMap();
240     for (Map.Entry<byte[],NavigableSet<byte[]>> entry : fams.entrySet()) {
241       byte [] fam = entry.getKey();
242       NavigableSet<byte[]> cols = entry.getValue();
243       if (cols != null && cols.size() > 0) {
244         for (byte[] col : cols) {
245           addColumn(fam, col);
246         }
247       } else {
248         addFamily(fam);
249       }
250     }
251     for (Map.Entry<String, byte[]> attr : scan.getAttributesMap().entrySet()) {
252       setAttribute(attr.getKey(), attr.getValue());
253     }
254   }
255 
256   /**
257    * Builds a scan object with the same specs as get.
258    * @param get get to model scan after
259    */
260   public Scan(Get get) {
261     this.startRow = get.getRow();
262     this.stopRow = get.getRow();
263     this.filter = get.getFilter();
264     this.cacheBlocks = get.getCacheBlocks();
265     this.maxVersions = get.getMaxVersions();
266     this.storeLimit = get.getMaxResultsPerColumnFamily();
267     this.storeOffset = get.getRowOffsetPerColumnFamily();
268     this.tr = get.getTimeRange();
269     this.familyMap = get.getFamilyMap();
270     this.getScan = true;
271     this.consistency = get.getConsistency();
272     this.setIsolationLevel(get.getIsolationLevel());
273     for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
274       setAttribute(attr.getKey(), attr.getValue());
275     }
276   }
277 
278   public boolean isGetScan() {
279     return this.getScan || isStartRowAndEqualsStopRow();
280   }
281 
282   private boolean isStartRowAndEqualsStopRow() {
283     return this.startRow != null && this.startRow.length > 0 &&
284         Bytes.equals(this.startRow, this.stopRow);
285   }
286   /**
287    * Get all columns from the specified family.
288    * <p>
289    * Overrides previous calls to addColumn for this family.
290    * @param family family name
291    * @return this
292    */
293   public Scan addFamily(byte [] family) {
294     familyMap.remove(family);
295     familyMap.put(family, null);
296     return this;
297   }
298 
299   /**
300    * Get the column from the specified family with the specified qualifier.
301    * <p>
302    * Overrides previous calls to addFamily for this family.
303    * @param family family name
304    * @param qualifier column qualifier
305    * @return this
306    */
307   public Scan addColumn(byte [] family, byte [] qualifier) {
308     NavigableSet<byte []> set = familyMap.get(family);
309     if(set == null) {
310       set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
311     }
312     if (qualifier == null) {
313       qualifier = HConstants.EMPTY_BYTE_ARRAY;
314     }
315     set.add(qualifier);
316     familyMap.put(family, set);
317     return this;
318   }
319 
320   /**
321    * Get versions of columns only within the specified timestamp range,
322    * [minStamp, maxStamp).  Note, default maximum versions to return is 1.  If
323    * your time range spans more than one version and you want all versions
324    * returned, up the number of versions beyond the default.
325    * @param minStamp minimum timestamp value, inclusive
326    * @param maxStamp maximum timestamp value, exclusive
327    * @throws IOException if invalid time range
328    * @see #setMaxVersions()
329    * @see #setMaxVersions(int)
330    * @return this
331    */
332   public Scan setTimeRange(long minStamp, long maxStamp)
333   throws IOException {
334     tr = new TimeRange(minStamp, maxStamp);
335     return this;
336   }
337 
338   /**
339    * Get versions of columns with the specified timestamp. Note, default maximum
340    * versions to return is 1.  If your time range spans more than one version
341    * and you want all versions returned, up the number of versions beyond the
342    * defaut.
343    * @param timestamp version timestamp
344    * @see #setMaxVersions()
345    * @see #setMaxVersions(int)
346    * @return this
347    */
348   public Scan setTimeStamp(long timestamp)
349   throws IOException {
350     try {
351       tr = new TimeRange(timestamp, timestamp+1);
352     } catch(IOException e) {
353       // This should never happen, unless integer overflow or something extremely wrong...
354       LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
355       throw e;
356     }
357     return this;
358   }
359 
360   /**
361    * Set the start row of the scan.
362    * @param startRow row to start scan on (inclusive)
363    * Note: In order to make startRow exclusive add a trailing 0 byte
364    * @return this
365    */
366   public Scan setStartRow(byte [] startRow) {
367     this.startRow = startRow;
368     return this;
369   }
370 
371   /**
372    * Set the stop row.
373    * @param stopRow row to end at (exclusive)
374    * <p><b>Note:</b> In order to make stopRow inclusive add a trailing 0 byte</p>
375    * <p><b>Note:</b> When doing a filter for a rowKey <u>Prefix</u>
376    * use {@link #setRowPrefixFilter(byte[])}.
377    * The 'trailing 0' will not yield the desired result.</p>
378    * @return this
379    */
380   public Scan setStopRow(byte [] stopRow) {
381     this.stopRow = stopRow;
382     return this;
383   }
384 
385   /**
386    * <p>Set a filter (using stopRow and startRow) so the result set only contains rows where the
387    * rowKey starts with the specified prefix.</p>
388    * <p>This is a utility method that converts the desired rowPrefix into the appropriate values
389    * for the startRow and stopRow to achieve the desired result.</p>
390    * <p>This can safely be used in combination with setFilter.</p>
391    * <p><b>NOTE: Doing a {@link #setStartRow(byte[])} and/or {@link #setStopRow(byte[])}
392    * after this method will yield undefined results.</b></p>
393    * @param rowPrefix the prefix all rows must start with. (Set <i>null</i> to remove the filter.)
394    * @return this
395    */
396   public Scan setRowPrefixFilter(byte[] rowPrefix) {
397     if (rowPrefix == null) {
398       setStartRow(HConstants.EMPTY_START_ROW);
399       setStopRow(HConstants.EMPTY_END_ROW);
400     } else {
401       this.setStartRow(rowPrefix);
402       this.setStopRow(calculateTheClosestNextRowKeyForPrefix(rowPrefix));
403     }
404     return this;
405   }
406 
407   /**
408    * <p>When scanning for a prefix the scan should stop immediately after the the last row that
409    * has the specified prefix. This method calculates the closest next rowKey immediately following
410    * the given rowKeyPrefix.</p>
411    * <p><b>IMPORTANT: This converts a rowKey<u>Prefix</u> into a rowKey</b>.</p>
412    * <p>If the prefix is an 'ASCII' string put into a byte[] then this is easy because you can
413    * simply increment the last byte of the array.
414    * But if your application uses real binary rowids you may run into the scenario that your
415    * prefix is something like:</p>
416    * &nbsp;&nbsp;&nbsp;<b>{ 0x12, 0x23, 0xFF, 0xFF }</b><br/>
417    * Then this stopRow needs to be fed into the actual scan<br/>
418    * &nbsp;&nbsp;&nbsp;<b>{ 0x12, 0x24 }</b> (Notice that it is shorter now)<br/>
419    * This method calculates the correct stop row value for this usecase.
420    *
421    * @param rowKeyPrefix the rowKey<u>Prefix</u>.
422    * @return the closest next rowKey immediately following the given rowKeyPrefix.
423    */
424   private byte[] calculateTheClosestNextRowKeyForPrefix(byte[] rowKeyPrefix) {
425     // Essentially we are treating it like an 'unsigned very very long' and doing +1 manually.
426     // Search for the place where the trailing 0xFFs start
427     int offset = rowKeyPrefix.length;
428     while (offset > 0) {
429       if (rowKeyPrefix[offset - 1] != (byte) 0xFF) {
430         break;
431       }
432       offset--;
433     }
434 
435     if (offset == 0) {
436       // We got an 0xFFFF... (only FFs) stopRow value which is
437       // the last possible prefix before the end of the table.
438       // So set it to stop at the 'end of the table'
439       return HConstants.EMPTY_END_ROW;
440     }
441 
442     // Copy the right length of the original
443     byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset);
444     // And increment the last one
445     newStopRow[newStopRow.length - 1]++;
446     return newStopRow;
447   }
448 
449   /**
450    * Get all available versions.
451    * @return this
452    */
453   public Scan setMaxVersions() {
454     this.maxVersions = Integer.MAX_VALUE;
455     return this;
456   }
457 
458   /**
459    * Get up to the specified number of versions of each column.
460    * @param maxVersions maximum versions for each column
461    * @return this
462    */
463   public Scan setMaxVersions(int maxVersions) {
464     this.maxVersions = maxVersions;
465     return this;
466   }
467 
468   /**
469    * Set the maximum number of values to return for each call to next().
470    * Callers should be aware that invoking this method with any value
471    * is equivalent to calling {@link #setAllowPartialResults(boolean)}
472    * with a value of {@code true}; partial results may be returned if
473    * this method is called. Use {@link #setMaxResultSize(long)}} to
474    * limit the size of a Scan's Results instead.
475    *
476    * @param batch the maximum number of values
477    */
478   public Scan setBatch(int batch) {
479     if (this.hasFilter() && this.filter.hasFilterRow()) {
480       throw new IncompatibleFilterException(
481         "Cannot set batch on a scan using a filter" +
482         " that returns true for filter.hasFilterRow");
483     }
484     this.batch = batch;
485     return this;
486   }
487 
488   /**
489    * Set the maximum number of values to return per row per Column Family
490    * @param limit the maximum number of values returned / row / CF
491    */
492   public Scan setMaxResultsPerColumnFamily(int limit) {
493     this.storeLimit = limit;
494     return this;
495   }
496 
497   /**
498    * Set offset for the row per Column Family.
499    * @param offset is the number of kvs that will be skipped.
500    */
501   public Scan setRowOffsetPerColumnFamily(int offset) {
502     this.storeOffset = offset;
503     return this;
504   }
505 
506   /**
507    * Set the number of rows for caching that will be passed to scanners.
508    * If not set, the Configuration setting {@link HConstants#HBASE_CLIENT_SCANNER_CACHING} will
509    * apply.
510    * Higher caching values will enable faster scanners but will use more memory.
511    * @param caching the number of rows for caching
512    */
513   public Scan setCaching(int caching) {
514     this.caching = caching;
515     return this;
516   }
517 
518   /**
519    * @return the maximum result size in bytes. See {@link #setMaxResultSize(long)}
520    */
521   public long getMaxResultSize() {
522     return maxResultSize;
523   }
524 
525   /**
526    * Set the maximum result size. The default is -1; this means that no specific
527    * maximum result size will be set for this scan, and the global configured
528    * value will be used instead. (Defaults to unlimited).
529    *
530    * @param maxResultSize The maximum result size in bytes.
531    */
532   public Scan setMaxResultSize(long maxResultSize) {
533     this.maxResultSize = maxResultSize;
534     return this;
535   }
536 
537   @Override
538   public Scan setFilter(Filter filter) {
539     super.setFilter(filter);
540     return this;
541   }
542 
543   /**
544    * Setting the familyMap
545    * @param familyMap map of family to qualifier
546    * @return this
547    */
548   public Scan setFamilyMap(Map<byte [], NavigableSet<byte []>> familyMap) {
549     this.familyMap = familyMap;
550     return this;
551   }
552 
553   /**
554    * Getting the familyMap
555    * @return familyMap
556    */
557   public Map<byte [], NavigableSet<byte []>> getFamilyMap() {
558     return this.familyMap;
559   }
560 
561   /**
562    * @return the number of families in familyMap
563    */
564   public int numFamilies() {
565     if(hasFamilies()) {
566       return this.familyMap.size();
567     }
568     return 0;
569   }
570 
571   /**
572    * @return true if familyMap is non empty, false otherwise
573    */
574   public boolean hasFamilies() {
575     return !this.familyMap.isEmpty();
576   }
577 
578   /**
579    * @return the keys of the familyMap
580    */
581   public byte[][] getFamilies() {
582     if(hasFamilies()) {
583       return this.familyMap.keySet().toArray(new byte[0][0]);
584     }
585     return null;
586   }
587 
588   /**
589    * @return the startrow
590    */
591   public byte [] getStartRow() {
592     return this.startRow;
593   }
594 
595   /**
596    * @return the stoprow
597    */
598   public byte [] getStopRow() {
599     return this.stopRow;
600   }
601 
602   /**
603    * @return the max number of versions to fetch
604    */
605   public int getMaxVersions() {
606     return this.maxVersions;
607   }
608 
609   /**
610    * @return maximum number of values to return for a single call to next()
611    */
612   public int getBatch() {
613     return this.batch;
614   }
615 
616   /**
617    * @return maximum number of values to return per row per CF
618    */
619   public int getMaxResultsPerColumnFamily() {
620     return this.storeLimit;
621   }
622 
623   /**
624    * Method for retrieving the scan's offset per row per column
625    * family (#kvs to be skipped)
626    * @return row offset
627    */
628   public int getRowOffsetPerColumnFamily() {
629     return this.storeOffset;
630   }
631 
632   /**
633    * @return caching the number of rows fetched when calling next on a scanner
634    */
635   public int getCaching() {
636     return this.caching;
637   }
638 
639   /**
640    * @return TimeRange
641    */
642   public TimeRange getTimeRange() {
643     return this.tr;
644   }
645 
646   /**
647    * @return RowFilter
648    */
649   @Override
650   public Filter getFilter() {
651     return filter;
652   }
653 
654   /**
655    * @return true is a filter has been specified, false if not
656    */
657   public boolean hasFilter() {
658     return filter != null;
659   }
660 
661   /**
662    * Set whether blocks should be cached for this Scan.
663    * <p>
664    * This is true by default.  When true, default settings of the table and
665    * family are used (this will never override caching blocks if the block
666    * cache is disabled for that family or entirely).
667    *
668    * @param cacheBlocks if false, default settings are overridden and blocks
669    * will not be cached
670    */
671   public Scan setCacheBlocks(boolean cacheBlocks) {
672     this.cacheBlocks = cacheBlocks;
673     return this;
674   }
675 
676   /**
677    * Get whether blocks should be cached for this Scan.
678    * @return true if default caching should be used, false if blocks should not
679    * be cached
680    */
681   public boolean getCacheBlocks() {
682     return cacheBlocks;
683   }
684 
685   /**
686    * Set whether this scan is a reversed one
687    * <p>
688    * This is false by default which means forward(normal) scan.
689    *
690    * @param reversed if true, scan will be backward order
691    * @return this
692    */
693   public Scan setReversed(boolean reversed) {
694     this.reversed = reversed;
695     return this;
696   }
697 
698   /**
699    * Get whether this scan is a reversed one.
700    * @return true if backward scan, false if forward(default) scan
701    */
702   public boolean isReversed() {
703     return reversed;
704   }
705 
706   /**
707    * Setting whether the caller wants to see the partial results that may be returned from the
708    * server. By default this value is false and the complete results will be assembled client side
709    * before being delivered to the caller.
710    * @param allowPartialResults
711    * @return this
712    */
713   public Scan setAllowPartialResults(final boolean allowPartialResults) {
714     this.allowPartialResults = allowPartialResults;
715     return this;
716   }
717 
718   /**
719    * @return true when the constructor of this scan understands that the results they will see may
720    *         only represent a partial portion of a row. The entire row would be retrieved by
721    *         subsequent calls to {@link ResultScanner#next()}
722    */
723   public boolean getAllowPartialResults() {
724     return allowPartialResults;
725   }
726 
727   /**
728    * Set the value indicating whether loading CFs on demand should be allowed (cluster
729    * default is false). On-demand CF loading doesn't load column families until necessary, e.g.
730    * if you filter on one column, the other column family data will be loaded only for the rows
731    * that are included in result, not all rows like in normal case.
732    * With column-specific filters, like SingleColumnValueFilter w/filterIfMissing == true,
733    * this can deliver huge perf gains when there's a cf with lots of data; however, it can
734    * also lead to some inconsistent results, as follows:
735    * - if someone does a concurrent update to both column families in question you may get a row
736    *   that never existed, e.g. for { rowKey = 5, { cat_videos => 1 }, { video => "my cat" } }
737    *   someone puts rowKey 5 with { cat_videos => 0 }, { video => "my dog" }, concurrent scan
738    *   filtering on "cat_videos == 1" can get { rowKey = 5, { cat_videos => 1 },
739    *   { video => "my dog" } }.
740    * - if there's a concurrent split and you have more than 2 column families, some rows may be
741    *   missing some column families.
742    */
743   public Scan setLoadColumnFamiliesOnDemand(boolean value) {
744     this.loadColumnFamiliesOnDemand = value;
745     return this;
746   }
747 
748   /**
749    * Get the raw loadColumnFamiliesOnDemand setting; if it's not set, can be null.
750    */
751   public Boolean getLoadColumnFamiliesOnDemandValue() {
752     return this.loadColumnFamiliesOnDemand;
753   }
754 
755   /**
756    * Get the logical value indicating whether on-demand CF loading should be allowed.
757    */
758   public boolean doLoadColumnFamiliesOnDemand() {
759     return (this.loadColumnFamiliesOnDemand != null)
760       && this.loadColumnFamiliesOnDemand.booleanValue();
761   }
762 
763   /**
764    * Compile the table and column family (i.e. schema) information
765    * into a String. Useful for parsing and aggregation by debugging,
766    * logging, and administration tools.
767    * @return Map
768    */
769   @Override
770   public Map<String, Object> getFingerprint() {
771     Map<String, Object> map = new HashMap<String, Object>();
772     List<String> families = new ArrayList<String>();
773     if(this.familyMap.size() == 0) {
774       map.put("families", "ALL");
775       return map;
776     } else {
777       map.put("families", families);
778     }
779     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
780         this.familyMap.entrySet()) {
781       families.add(Bytes.toStringBinary(entry.getKey()));
782     }
783     return map;
784   }
785 
786   /**
787    * Compile the details beyond the scope of getFingerprint (row, columns,
788    * timestamps, etc.) into a Map along with the fingerprinted information.
789    * Useful for debugging, logging, and administration tools.
790    * @param maxCols a limit on the number of columns output prior to truncation
791    * @return Map
792    */
793   @Override
794   public Map<String, Object> toMap(int maxCols) {
795     // start with the fingerpring map and build on top of it
796     Map<String, Object> map = getFingerprint();
797     // map from families to column list replaces fingerprint's list of families
798     Map<String, List<String>> familyColumns =
799       new HashMap<String, List<String>>();
800     map.put("families", familyColumns);
801     // add scalar information first
802     map.put("startRow", Bytes.toStringBinary(this.startRow));
803     map.put("stopRow", Bytes.toStringBinary(this.stopRow));
804     map.put("maxVersions", this.maxVersions);
805     map.put("batch", this.batch);
806     map.put("caching", this.caching);
807     map.put("maxResultSize", this.maxResultSize);
808     map.put("cacheBlocks", this.cacheBlocks);
809     map.put("loadColumnFamiliesOnDemand", this.loadColumnFamiliesOnDemand);
810     List<Long> timeRange = new ArrayList<Long>();
811     timeRange.add(this.tr.getMin());
812     timeRange.add(this.tr.getMax());
813     map.put("timeRange", timeRange);
814     int colCount = 0;
815     // iterate through affected families and list out up to maxCols columns
816     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
817       this.familyMap.entrySet()) {
818       List<String> columns = new ArrayList<String>();
819       familyColumns.put(Bytes.toStringBinary(entry.getKey()), columns);
820       if(entry.getValue() == null) {
821         colCount++;
822         --maxCols;
823         columns.add("ALL");
824       } else {
825         colCount += entry.getValue().size();
826         if (maxCols <= 0) {
827           continue;
828         }
829         for (byte [] column : entry.getValue()) {
830           if (--maxCols <= 0) {
831             continue;
832           }
833           columns.add(Bytes.toStringBinary(column));
834         }
835       }
836     }
837     map.put("totalColumns", colCount);
838     if (this.filter != null) {
839       map.put("filter", this.filter.toString());
840     }
841     // add the id if set
842     if (getId() != null) {
843       map.put("id", getId());
844     }
845     return map;
846   }
847 
848   /**
849    * Enable/disable "raw" mode for this scan.
850    * If "raw" is enabled the scan will return all
851    * delete marker and deleted rows that have not
852    * been collected, yet.
853    * This is mostly useful for Scan on column families
854    * that have KEEP_DELETED_ROWS enabled.
855    * It is an error to specify any column when "raw" is set.
856    * @param raw True/False to enable/disable "raw" mode.
857    */
858   public Scan setRaw(boolean raw) {
859     setAttribute(RAW_ATTR, Bytes.toBytes(raw));
860     return this;
861   }
862 
863   /**
864    * @return True if this Scan is in "raw" mode.
865    */
866   public boolean isRaw() {
867     byte[] attr = getAttribute(RAW_ATTR);
868     return attr == null ? false : Bytes.toBoolean(attr);
869   }
870 
871 
872 
873   /**
874    * Set whether this scan is a small scan
875    * <p>
876    * Small scan should use pread and big scan can use seek + read
877    *
878    * seek + read is fast but can cause two problem (1) resource contention (2)
879    * cause too much network io
880    *
881    * [89-fb] Using pread for non-compaction read request
882    * https://issues.apache.org/jira/browse/HBASE-7266
883    *
884    * On the other hand, if setting it true, we would do
885    * openScanner,next,closeScanner in one RPC call. It means the better
886    * performance for small scan. [HBASE-9488].
887    *
888    * Generally, if the scan range is within one data block(64KB), it could be
889    * considered as a small scan.
890    *
891    * @param small
892    */
893   public Scan setSmall(boolean small) {
894     this.small = small;
895     return this;
896   }
897 
898   /**
899    * Get whether this scan is a small scan
900    * @return true if small scan
901    */
902   public boolean isSmall() {
903     return small;
904   }
905 
906   @Override
907   public Scan setAttribute(String name, byte[] value) {
908     return (Scan) super.setAttribute(name, value);
909   }
910 
911   @Override
912   public Scan setId(String id) {
913     return (Scan) super.setId(id);
914   }
915 
916   @Override
917   public Scan setAuthorizations(Authorizations authorizations) {
918     return (Scan) super.setAuthorizations(authorizations);
919   }
920 
921   @Override
922   public Scan setACL(Map<String, Permission> perms) {
923     return (Scan) super.setACL(perms);
924   }
925 
926   @Override
927   public Scan setACL(String user, Permission perms) {
928     return (Scan) super.setACL(user, perms);
929   }
930 
931   @Override
932   public Scan setConsistency(Consistency consistency) {
933     return (Scan) super.setConsistency(consistency);
934   }
935 
936   @Override
937   public Scan setReplicaId(int Id) {
938     return (Scan) super.setReplicaId(Id);
939   }
940 
941   @Override
942   public Scan setIsolationLevel(IsolationLevel level) {
943     return (Scan) super.setIsolationLevel(level);
944   }
945 
946   /**
947    * Utility that creates a Scan that will do a  small scan in reverse from passed row
948    * looking for next closest row.
949    * @param row
950    * @param family
951    * @return An instance of Scan primed with passed <code>row</code> and <code>family</code> to
952    * scan in reverse for one row only.
953    */
954   static Scan createGetClosestRowOrBeforeReverseScan(byte[] row) {
955     // Below does not work if you add in family; need to add the family qualifier that is highest
956     // possible family qualifier.  Do we have such a notion?  Would have to be magic.
957     Scan scan = new Scan(row);
958     scan.setSmall(true);
959     scan.setReversed(true);
960     scan.setCaching(1);
961     return scan;
962   }
963 
964   /**
965    * Enable collection of {@link ScanMetrics}. For advanced users.
966    * @param enabled Set to true to enable accumulating scan metrics
967    */
968   public Scan setScanMetricsEnabled(final boolean enabled) {
969     setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.valueOf(enabled)));
970     return this;
971   }
972 
973   /**
974    * @return True if collection of scan metrics is enabled. For advanced users.
975    */
976   public boolean isScanMetricsEnabled() {
977     byte[] attr = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
978     return attr == null ? false : Bytes.toBoolean(attr);
979   }
980 
981   /**
982    * @return Metrics on this Scan, if metrics were enabled.
983    * @see #setScanMetricsEnabled(boolean)
984    */
985   public ScanMetrics getScanMetrics() {
986     byte [] bytes = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
987     if (bytes == null) return null;
988     return ProtobufUtil.toScanMetrics(bytes);
989   }
990 }