Class Scan
- All Implemented Interfaces:
- Attributes
- Direct Known Subclasses:
- ImmutableScan,- InternalScan
 All operations are identical to Get with the exception of instantiation. Rather than
 specifying a single row, an optional startRow and stopRow may be defined. If rows are not
 specified, the Scanner will iterate over all rows.
 
 To get all columns from all rows of a Table, create an instance with no constraints; use the
 Scan() constructor. To constrain the scan to specific column families, call
 addFamily for each family to retrieve on your Scan instance.
 
 To get specific columns, call addColumn for each column to
 retrieve.
 
 To only retrieve columns within a specific range of version timestamps, call
 setTimeRange.
 
 To only retrieve columns with a specific timestamp, call setTimestamp
 .
 
 To limit the number of versions of each column to be returned, call readVersions(int).
 
 To limit the maximum number of values returned for each call to next(), call
 setBatch.
 
 To add a filter, call setFilter.
 
 For small scan, it is deprecated in 2.0.0. Now we have a setLimit(int) method in Scan
 object which is used to tell RS how many rows we want. If the rows return reaches the limit, the
 RS will close the RegionScanner automatically. And we will also fetch data when openScanner in
 the new implementation, this means we can also finish a scan operation in one rpc call. And we
 have also introduced a setReadType(ReadType) method. You can use this method to tell RS
 to use pread explicitly.
 
 Expert: To explicitly disable server-side block caching for this scan, execute
 setCacheBlocks(boolean).
 
Note: Usage alters Scan instances. Internally, attributes are updated as the Scan runs and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when you go to clone a Scan instance or if you go to reuse a created Scan instance; safer is create a Scan instance per usage.
- 
Nested Class SummaryNested Classes
- 
Field SummaryFieldsModifier and TypeFieldDescriptionprivate booleanprivate Booleanprivate intprivate booleanprivate int-1 means no caching specified and the value ofHConstants.HBASE_CLIENT_SCANNER_CACHING(default toHConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING) will be usedstatic final booleanDefault value ofHBASE_CLIENT_SCANNER_ASYNC_PREFETCH.private Map<byte[],NavigableSet<byte[]>> static final StringParameter name for client scanner sync/async prefetch toggle.private booleanprivate booleanprivate intThe number of rows we want for this scan.private static final org.slf4j.Loggerprivate longprivate intprivate longThe mvcc read point to use when open a scanner.private booleanprivate static final Stringprivate Scan.ReadTypeControl whether to use pread at server side.private booleanprivate static final Stringprivate static final Stringstatic final Stringprivate byte[]private byte[]private intprivate intprivate TimeRangeFields inherited from class org.apache.hadoop.hbase.client.QuerycolFamTimeRangeMap, consistency, filter, loadColumnFamiliesOnDemand, queryMetricsEnabled, targetReplicaIdFields inherited from class org.apache.hadoop.hbase.client.OperationWithAttributesID_ATRIBUTE
- 
Constructor SummaryConstructors
- 
Method SummaryModifier and TypeMethodDescriptionaddColumn(byte[] family, byte[] qualifier) Get the column from the specified family with the specified qualifier.addFamily(byte[] family) Get all columns from the specified family.static ScancreateScanFromCursor(Cursor cursor) Create a new Scan with a cursor.booleanReturns true when the constructor of this scan understands that the results they will see may only represent a partial portion of a row.intgetBatch()Returns maximum number of values to return for a single call to next()booleanGet whether blocks should be cached for this Scan.intReturns caching the number of rows fetched when calling next on a scannerbyte[][]Returns the keys of the familyMapMap<byte[],NavigableSet<byte[]>> Getting the familyMapReturns RowFilterCompile the table and column family (i.e.intgetLimit()Returns the limit of rows for this scanlongReturns the maximum result size in bytes.intReturns maximum number of values to return per row per CFintReturns the max number of versions to fetch(package private) longGet the mvcc read point used to open a scanner.Returns the read type for this scanintMethod for retrieving the scan's offset per row per column family (#kvs to be skipped)byte[]Returns the startrowbyte[]Returns the stoprowReturns TimeRangebooleanReturns true if familyMap is non empty, false otherwisebooleanReturns true is a filter has been specified, false if notbooleanReturns if we should include start row when scanbooleanReturns if we should include stop row when scanbooleanbooleanbooleanisRaw()Returns True if this Scan is in "raw" mode.booleanGet whether this scan is a reversed one.booleanbooleanReturns True if collection of scan metrics is enabled.intReturns the number of families in familyMapGet all available versions.readVersions(int versions) Get up to the specified number of versions of each column.(package private) ScanSet the mvcc read point to -1 which means do not use it.setACL(String user, Permission perms) Set the ACL for the operation.setACL(Map<String, Permission> perms) Set the ACL for the operation.setAllowPartialResults(boolean allowPartialResults) Setting whether the caller wants to see the partial results when server returns less-than-expected cells.setAsyncPrefetch(boolean asyncPrefetch) Deprecated.Since 3.0.0, will be removed in 4.0.0.setAttribute(String name, byte[] value) Sets an attribute.setAuthorizations(Authorizations authorizations) Sets the authorizations to be used by this QuerysetBatch(int batch) Set the maximum number of cells to return for each call to next().setCacheBlocks(boolean cacheBlocks) Set whether blocks should be cached for this Scan.setCaching(int caching) Set the number of rows for caching that will be passed to scanners.setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) Get versions of columns only within the specified timestamp range, [minStamp, maxStamp) on a per CF bases.setConsistency(Consistency consistency) Sets the consistency level for this operationsetEnableScanMetricsByRegion(boolean enable) Enables region level scan metrics.setFamilyMap(Map<byte[], NavigableSet<byte[]>> familyMap) Setting the familyMapApply the specified server-side filter when performing the Query.This method allows you to set an identifier on an operation.setIsolationLevel(IsolationLevel level) Set the isolation level for this query.setLimit(int limit) Set the limit of rows for this scan.setLoadColumnFamiliesOnDemand(boolean value) Set the value indicating whether loading CFs on demand should be allowed (cluster default is false).setMaxResultSize(long maxResultSize) Set the maximum result size.setMaxResultsPerColumnFamily(int limit) Set the maximum number of values to return per row per Column Family(package private) ScansetMvccReadPoint(long mvccReadPoint) Set the mvcc read point used to open a scanner.setNeedCursorResult(boolean needCursorResult) When the server is slow or we scan a table with many deleted data or we use a sparse filter, the server will response heartbeat to prevent timeout.Call this when you only want to get one row.setPriority(int priority) setRaw(boolean raw) Enable/disable "raw" mode for this scan.setReadType(Scan.ReadType readType) Set the read type for this scan.setReplicaId(int Id) Specify region replica id where Query will fetch data from.setReversed(boolean reversed) Set whether this scan is a reversed onesetRowOffsetPerColumnFamily(int offset) Set offset for the row per Column Family.setRowPrefixFilter(byte[] rowPrefix) Deprecated.since 2.5.0, will be removed in 4.0.0.setScanMetricsEnabled(boolean enabled) Enable collection ofScanMetrics.setStartStopRowForPrefixScan(byte[] rowPrefix) Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey starts with the specified prefix.setTimeRange(long minStamp, long maxStamp) Get versions of columns only within the specified timestamp range, [minStamp, maxStamp).setTimestamp(long timestamp) Get versions of columns with the specified timestamp.toMap(int maxCols) Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a Map along with the fingerprinted information.withStartRow(byte[] startRow) Set the start row of the scan.withStartRow(byte[] startRow, boolean inclusive) Set the start row of the scan.withStopRow(byte[] stopRow) Set the stop row of the scan.withStopRow(byte[] stopRow, boolean inclusive) Set the stop row of the scan.Methods inherited from class org.apache.hadoop.hbase.client.QuerydoLoadColumnFamiliesOnDemand, getACL, getAuthorizations, getColumnFamilyTimeRange, getConsistency, getIsolationLevel, getLoadColumnFamiliesOnDemandValue, getReplicaId, isQueryMetricsEnabled, setQueryMetricsEnabledMethods inherited from class org.apache.hadoop.hbase.client.OperationWithAttributesgetAttribute, getAttributeSize, getAttributesMap, getId, getPriority
- 
Field Details- 
LOG
- 
RAW_ATTR- See Also:
 
- 
startRow
- 
includeStartRow
- 
stopRow
- 
includeStopRow
- 
maxVersions
- 
batch
- 
allowPartialResultsPartialResults areResults must be combined to form a completeResult. TheResults had to be returned in fragments (i.e. as partials) because the size of the cells in the row exceeded max result size on the server. Typically partial results will be combined client side into complete results before being delivered to the caller. However, if this flag is set, the caller is indicating that they do not mind seeing partial results (i.e. they understand that the results returned from the Scanner may only represent part of a particular row). In such a case, any attempt to combine the partials into a complete result on the client side will be skipped, and the caller will be able to see the exact results returned from the server.
- 
storeLimit
- 
storeOffset
- 
SCAN_ATTRIBUTES_METRICS_ENABLE- See Also:
 
- 
SCAN_ATTRIBUTES_TABLE_NAME- See Also:
 
- 
SCAN_ATTRIBUTES_METRICS_BY_REGION_ENABLE- See Also:
 
- 
caching-1 means no caching specified and the value ofHConstants.HBASE_CLIENT_SCANNER_CACHING(default toHConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING) will be used
- 
maxResultSize
- 
cacheBlocks
- 
reversed
- 
tr
- 
familyMap
- 
asyncPrefetch
- 
HBASE_CLIENT_SCANNER_ASYNC_PREFETCHParameter name for client scanner sync/async prefetch toggle. When using async scanner, prefetching data from the server is done at the background. The parameter currently won't have any effect in the case that the user has set Scan#setSmall or Scan#setReversed- See Also:
 
- 
DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCHDefault value ofHBASE_CLIENT_SCANNER_ASYNC_PREFETCH.- See Also:
 
- 
mvccReadPointThe mvcc read point to use when open a scanner. Remember to clear it after switching regions as the mvcc is only valid within region scope.
- 
limitThe number of rows we want for this scan. We will terminate the scan if the number of return rows reaches this value.
- 
readTypeControl whether to use pread at server side.
- 
needCursorResult
 
- 
- 
Constructor Details- 
Scanpublic Scan()Create a Scan operation across all rows.
- 
ScanCreates a new instance of this class while copying all values.- Parameters:
- scan- The scan instance to copy from.
- Throws:
- IOException- When copying the values fails.
 
- 
ScanBuilds a scan object with the same specs as get.- Parameters:
- get- get to model scan after
 
 
- 
- 
Method Details- 
isGetScan
- 
addFamilyGet all columns from the specified family.Overrides previous calls to addColumn for this family. - Parameters:
- family- family name
 
- 
addColumnGet the column from the specified family with the specified qualifier.Overrides previous calls to addFamily for this family. - Parameters:
- family- family name
- qualifier- column qualifier
 
- 
setTimeRangeGet versions of columns only within the specified timestamp range, [minStamp, maxStamp). Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the default.- Parameters:
- minStamp- minimum timestamp value, inclusive
- maxStamp- maximum timestamp value, exclusive
- Throws:
- IOException
- See Also:
 
- 
setTimestampGet versions of columns with the specified timestamp. Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the defaut.- Parameters:
- timestamp- version timestamp
- See Also:
 
- 
setColumnFamilyTimeRangeDescription copied from class:QueryGet versions of columns only within the specified timestamp range, [minStamp, maxStamp) on a per CF bases. Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the default. Column Family time ranges take precedence over the global time range.- Overrides:
- setColumnFamilyTimeRangein class- Query
- Parameters:
- cf- the column family for which you want to restrict
- minStamp- minimum timestamp value, inclusive
- maxStamp- maximum timestamp value, exclusive
 
- 
withStartRowSet the start row of the scan.If the specified row does not exist, the Scanner will start from the next closest row after the specified row. Note: Do NOT use this in combination with setRowPrefixFilter(byte[])orsetStartStopRowForPrefixScan(byte[]). Doing so will make the scan result unexpected or even undefined.- Parameters:
- startRow- row to start scanner at or after
- Throws:
- IllegalArgumentException- if startRow does not meet criteria for a row key (when length exceeds- HConstants.MAX_ROW_LENGTH)
 
- 
withStartRowSet the start row of the scan.If the specified row does not exist, or the inclusiveisfalse, the Scanner will start from the next closest row after the specified row.Note: Do NOT use this in combination with setRowPrefixFilter(byte[])orsetStartStopRowForPrefixScan(byte[]). Doing so will make the scan result unexpected or even undefined.- Parameters:
- startRow- row to start scanner at or after
- inclusive- whether we should include the start row when scan
- Throws:
- IllegalArgumentException- if startRow does not meet criteria for a row key (when length exceeds- HConstants.MAX_ROW_LENGTH)
 
- 
withStopRowSet the stop row of the scan.The scan will include rows that are lexicographically less than the provided stopRow. Note: Do NOT use this in combination with setRowPrefixFilter(byte[])orsetStartStopRowForPrefixScan(byte[]). Doing so will make the scan result unexpected or even undefined.- Parameters:
- stopRow- row to end at (exclusive)
- Throws:
- IllegalArgumentException- if stopRow does not meet criteria for a row key (when length exceeds- HConstants.MAX_ROW_LENGTH)
 
- 
withStopRowSet the stop row of the scan.The scan will include rows that are lexicographically less than (or equal to if inclusiveistrue) the provided stopRow.Note: Do NOT use this in combination with setRowPrefixFilter(byte[])orsetStartStopRowForPrefixScan(byte[]). Doing so will make the scan result unexpected or even undefined.- Parameters:
- stopRow- row to end at
- inclusive- whether we should include the stop row when scan
- Throws:
- IllegalArgumentException- if stopRow does not meet criteria for a row key (when length exceeds- HConstants.MAX_ROW_LENGTH)
 
- 
setRowPrefixFilterDeprecated.since 2.5.0, will be removed in 4.0.0. The name of this method is considered to be confusing as it does not use aFilterbut uses setting the startRow and stopRow instead. UsesetStartStopRowForPrefixScan(byte[])instead.Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey starts with the specified prefix. This is a utility method that converts the desired rowPrefix into the appropriate values for the startRow and stopRow to achieve the desired result. This can safely be used in combination with setFilter. This CANNOT be used in combination with withStartRow and/or withStopRow. Such a combination will yield unexpected and even undefined results. - Parameters:
- rowPrefix- the prefix all rows must start with. (Set null to remove the filter.)
 
- 
setStartStopRowForPrefixScanSet a filter (using stopRow and startRow) so the result set only contains rows where the rowKey starts with the specified prefix. This is a utility method that converts the desired rowPrefix into the appropriate values for the startRow and stopRow to achieve the desired result. This can safely be used in combination with setFilter. This CANNOT be used in combination with withStartRow and/or withStopRow. Such a combination will yield unexpected and even undefined results. - Parameters:
- rowPrefix- the prefix all rows must start with. (Set null to remove the filter.)
 
- 
readAllVersionsGet all available versions.
- 
readVersionsGet up to the specified number of versions of each column.- Parameters:
- versions- specified number of versions for each column
 
- 
setBatchSet the maximum number of cells to return for each call to next(). Callers should be aware that this is not equivalent to callingsetAllowPartialResults(boolean). If you don't allow partial results, the number of cells in each Result must equal to your batch setting unless it is the last Result for current row. So this method is helpful in paging queries. If you just want to prevent OOM at client, use setAllowPartialResults(true) is better.- Parameters:
- batch- the maximum number of values
- See Also:
 
- 
setMaxResultsPerColumnFamilySet the maximum number of values to return per row per Column Family- Parameters:
- limit- the maximum number of values returned / row / CF
 
- 
setRowOffsetPerColumnFamilySet offset for the row per Column Family.- Parameters:
- offset- is the number of kvs that will be skipped.
 
- 
setCachingSet the number of rows for caching that will be passed to scanners. If not set, the Configuration settingHConstants.HBASE_CLIENT_SCANNER_CACHINGwill apply. Higher caching values will enable faster scanners but will use more memory.- Parameters:
- caching- the number of rows for caching
 
- 
getMaxResultSizeReturns the maximum result size in bytes. SeesetMaxResultSize(long)
- 
setMaxResultSizeSet the maximum result size. The default is -1; this means that no specific maximum result size will be set for this scan, and the global configured value will be used instead. (Defaults to unlimited).- Parameters:
- maxResultSize- The maximum result size in bytes.
 
- 
setFilterDescription copied from class:QueryApply the specified server-side filter when performing the Query. OnlyFilter.filterCell(org.apache.hadoop.hbase.Cell)is called AFTER all tests for ttl, column match, deletes and column family's max versions have been run.
- 
setFamilyMapSetting the familyMap- Parameters:
- familyMap- map of family to qualifier
 
- 
getFamilyMapGetting the familyMap
- 
numFamiliesReturns the number of families in familyMap
- 
hasFamiliesReturns true if familyMap is non empty, false otherwise
- 
getFamiliesReturns the keys of the familyMap
- 
getStartRowReturns the startrow
- 
includeStartRowReturns if we should include start row when scan
- 
getStopRowReturns the stoprow
- 
includeStopRowReturns if we should include stop row when scan
- 
getMaxVersionsReturns the max number of versions to fetch
- 
getBatchReturns maximum number of values to return for a single call to next()
- 
getMaxResultsPerColumnFamilyReturns maximum number of values to return per row per CF
- 
getRowOffsetPerColumnFamilyMethod for retrieving the scan's offset per row per column family (#kvs to be skipped)- Returns:
- row offset
 
- 
getCachingReturns caching the number of rows fetched when calling next on a scanner
- 
getTimeRangeReturns TimeRange
- 
getFilterReturns RowFilter
- 
hasFilterReturns true is a filter has been specified, false if not
- 
setCacheBlocksSet whether blocks should be cached for this Scan.This is true by default. When true, default settings of the table and family are used (this will never override caching blocks if the block cache is disabled for that family or entirely). - Parameters:
- cacheBlocks- if false, default settings are overridden and blocks will not be cached
 
- 
getCacheBlocksGet whether blocks should be cached for this Scan.- Returns:
- true if default caching should be used, false if blocks should not be cached
 
- 
setReversedSet whether this scan is a reversed oneThis is false by default which means forward(normal) scan. - Parameters:
- reversed- if true, scan will be backward order
 
- 
isReversedGet whether this scan is a reversed one.- Returns:
- true if backward scan, false if forward(default) scan
 
- 
setAllowPartialResultsSetting whether the caller wants to see the partial results when server returns less-than-expected cells. It is helpful while scanning a huge row to prevent OOM at client. By default this value is false and the complete results will be assembled client side before being delivered to the caller.- See Also:
 
- 
getAllowPartialResultsReturns true when the constructor of this scan understands that the results they will see may only represent a partial portion of a row. The entire row would be retrieved by subsequent calls toResultScanner.next()
- 
setLoadColumnFamiliesOnDemandDescription copied from class:QuerySet the value indicating whether loading CFs on demand should be allowed (cluster default is false). On-demand CF loading doesn't load column families until necessary, e.g. if you filter on one column, the other column family data will be loaded only for the rows that are included in result, not all rows like in normal case. With column-specific filters, like SingleColumnValueFilter w/filterIfMissing == true, this can deliver huge perf gains when there's a cf with lots of data; however, it can also lead to some inconsistent results, as follows: - if someone does a concurrent update to both column families in question you may get a row that never existed, e.g. for { rowKey = 5, { cat_videos => 1 }, { video => "my cat" } } someone puts rowKey 5 with { cat_videos => 0 }, { video => "my dog" }, concurrent scan filtering on "cat_videos == 1" can get { rowKey = 5, { cat_videos => 1 }, { video => "my dog" } }. - if there's a concurrent split and you have more than 2 column families, some rows may be missing some column families.- Overrides:
- setLoadColumnFamiliesOnDemandin class- Query
 
- 
getFingerprintCompile the table and column family (i.e. schema) information into a String. Useful for parsing and aggregation by debugging, logging, and administration tools.- Specified by:
- getFingerprintin class- Operation
- Returns:
- a map containing fingerprint information (i.e. column families)
 
- 
toMapCompile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a Map along with the fingerprinted information. Useful for debugging, logging, and administration tools.
- 
setRawEnable/disable "raw" mode for this scan. If "raw" is enabled the scan will return all delete marker and deleted rows that have not been collected, yet. This is mostly useful for Scan on column families that have KEEP_DELETED_ROWS enabled. It is an error to specify any column when "raw" is set.- Parameters:
- raw- True/False to enable/disable "raw" mode.
 
- 
isRawReturns True if this Scan is in "raw" mode.
- 
setAttributeDescription copied from interface:AttributesSets an attribute. In case value = null attribute is removed from the attributes map. Attribute names starting with _ indicate system attributes.- Specified by:
- setAttributein interface- Attributes
- Overrides:
- setAttributein class- OperationWithAttributes
- Parameters:
- name- attribute name
- value- attribute value
 
- 
setIdDescription copied from class:OperationWithAttributesThis method allows you to set an identifier on an operation. The original motivation for this was to allow the identifier to be used in slow query logging, but this could obviously be useful in other places. One use of this could be to put a class.method identifier in here to see where the slow query is coming from. id to set for the scan- Overrides:
- setIdin class- OperationWithAttributes
 
- 
setAuthorizationsDescription copied from class:QuerySets the authorizations to be used by this Query- Overrides:
- setAuthorizationsin class- Query
 
- 
setACLDescription copied from class:QuerySet the ACL for the operation.
- 
setACLDescription copied from class:QuerySet the ACL for the operation.
- 
setConsistencyDescription copied from class:QuerySets the consistency level for this operation- Overrides:
- setConsistencyin class- Query
- Parameters:
- consistency- the consistency level
 
- 
setReplicaIdDescription copied from class:QuerySpecify region replica id where Query will fetch data from. Use this together withQuery.setConsistency(Consistency)passingConsistency.TIMELINEto read data from a specific replicaId.
 Expert: This is an advanced API exposed. Only use it if you know what you are doing- Overrides:
- setReplicaIdin class- Query
 
- 
setIsolationLevelDescription copied from class:QuerySet the isolation level for this query. If the isolation level is set to READ_UNCOMMITTED, then this query will return data from committed and uncommitted transactions. If the isolation level is set to READ_COMMITTED, then this query will return data from committed transactions only. If a isolation level is not explicitly set on a Query, then it is assumed to be READ_COMMITTED.- Overrides:
- setIsolationLevelin class- Query
- Parameters:
- level- IsolationLevel for this query
 
- 
setPriority- Overrides:
- setPriorityin class- OperationWithAttributes
 
- 
setScanMetricsEnabledEnable collection ofScanMetrics. For advanced users. While disabling scan metrics, will also disable region level scan metrics.- Parameters:
- enabled- Set to true to enable accumulating scan metrics
 
- 
isScanMetricsEnabledReturns True if collection of scan metrics is enabled. For advanced users.
- 
isAsyncPrefetch
- 
setAsyncPrefetchDeprecated.Since 3.0.0, will be removed in 4.0.0. After building sync client upon async client, the implementation is always 'async prefetch', so this flag is useless now.
- 
getLimitReturns the limit of rows for this scan
- 
setLimitSet the limit of rows for this scan. We will terminate the scan if the number of returned rows reaches this value.This condition will be tested at last, after all other conditions such as stopRow, filter, etc. - Parameters:
- limit- the limit of rows for this scan
 
- 
setOneRowLimitCall this when you only want to get one row. It will setlimitto1, and also setreadTypetoScan.ReadType.PREAD.
- 
getReadTypeReturns the read type for this scan
- 
setReadTypeSet the read type for this scan.Notice that we may choose to use pread even if you specific Scan.ReadType.STREAMhere. For example, we will always use pread if this is a get scan.
- 
getMvccReadPointlong getMvccReadPoint()Get the mvcc read point used to open a scanner.
- 
setMvccReadPointSet the mvcc read point used to open a scanner.
- 
resetMvccReadPointSet the mvcc read point to -1 which means do not use it.
- 
setNeedCursorResultWhen the server is slow or we scan a table with many deleted data or we use a sparse filter, the server will response heartbeat to prevent timeout. However the scanner will return a Result only when client can do it. So if there are many heartbeats, the blocking time on ResultScanner#next() may be very long, which is not friendly to online services. Set this to true then you can get a special Result whose #isCursor() returns true and is not contains any real data. It only tells you where the server has scanned. You can call next to continue scanning or open a new scanner with this row key as start row whenever you want. Users can get a cursor when and only when there is a response from the server but we can not return a Result to users, for example, this response is a heartbeat or there are partial cells but users do not allow partial result. Now the cursor is in row level which means the special Result will only contains a row key.Result.isCursor()Result.getCursor()Cursor
- 
isNeedCursorResult
- 
createScanFromCursorCreate a new Scan with a cursor. It only set the position information like start row key. The others (like cfs, stop row, limit) should still be filled in by the user.Result.isCursor()Result.getCursor()Cursor
- 
setEnableScanMetricsByRegionEnables region level scan metrics. If scan metrics are disabled then first enables scan metrics followed by region level scan metrics.- Parameters:
- enable- Set to true to enable region level scan metrics.
 
- 
isScanMetricsByRegionEnabled
 
-