@InterfaceAudience.Public public class Scan extends Query
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 setMaxVersions
.
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.
Modifier and Type | Class and Description |
---|---|
static class |
Scan.ReadType |
Modifier and Type | Field and Description |
---|---|
private boolean |
allowPartialResults
|
private Boolean |
asyncPrefetch |
private int |
batch |
private boolean |
cacheBlocks |
private int |
caching
-1 means no caching specified and the value of
HConstants.HBASE_CLIENT_SCANNER_CACHING
(default to HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING ) will be used |
static boolean |
DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH
Default value of
HBASE_CLIENT_SCANNER_ASYNC_PREFETCH . |
private Map<byte[],NavigableSet<byte[]>> |
familyMap |
static String |
HBASE_CLIENT_SCANNER_ASYNC_PREFETCH
Parameter name for client scanner sync/async prefetch toggle.
|
private boolean |
includeStartRow |
private boolean |
includeStopRow |
private int |
limit
The number of rows we want for this scan.
|
private static org.slf4j.Logger |
LOG |
private long |
maxResultSize |
private int |
maxVersions |
private long |
mvccReadPoint
The mvcc read point to use when open a scanner.
|
private boolean |
needCursorResult |
private static String |
RAW_ATTR |
private Scan.ReadType |
readType
Control whether to use pread at server side.
|
private boolean |
reversed |
static String |
SCAN_ATTRIBUTES_METRICS_DATA
Deprecated.
|
static String |
SCAN_ATTRIBUTES_METRICS_ENABLE
Deprecated.
since 1.0.0. Use
setScanMetricsEnabled(boolean) |
static String |
SCAN_ATTRIBUTES_TABLE_NAME |
private boolean |
small
Set it true for small scan to get better performance Small scan should use pread and big scan
can use seek + read seek + read is fast but can cause two problem (1) resource contention (2)
cause too much network io [89-fb] Using pread for non-compaction read request
https://issues.apache.org/jira/browse/HBASE-7266 On the other hand, if setting it true, we
would do openScanner,next,closeScanner in one RPC call.
|
private byte[] |
startRow |
private byte[] |
stopRow |
private int |
storeLimit |
private int |
storeOffset |
private TimeRange |
tr |
colFamTimeRangeMap, consistency, filter, loadColumnFamiliesOnDemand, targetReplicaId
ID_ATRIBUTE
Constructor and Description |
---|
Scan()
Create a Scan operation across all rows.
|
Scan(byte[] startRow)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
new Scan().withStartRow(startRow) instead. |
Scan(byte[] startRow,
byte[] stopRow)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
new Scan().withStartRow(startRow).withStopRow(stopRow) instead. |
Scan(byte[] startRow,
Filter filter)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
new Scan().withStartRow(startRow).setFilter(filter) instead. |
Scan(Get get)
Builds a scan object with the same specs as get.
|
Scan(Scan scan)
Creates a new instance of this class while copying all values.
|
Modifier and Type | Method and Description |
---|---|
Scan |
addColumn(byte[] family,
byte[] qualifier)
Get the column from the specified family with the specified qualifier.
|
Scan |
addFamily(byte[] family)
Get all columns from the specified family.
|
static Scan |
createScanFromCursor(Cursor cursor)
Create a new Scan with a cursor.
|
boolean |
getAllowPartialResults()
Returns true when the constructor of this scan understands that the results they will see may
only represent a partial portion of a row.
|
int |
getBatch()
Returns maximum number of values to return for a single call to next()
|
boolean |
getCacheBlocks()
Get whether blocks should be cached for this Scan.
|
int |
getCaching()
Returns caching the number of rows fetched when calling next on a scanner
|
byte[][] |
getFamilies()
Returns the keys of the familyMap
|
Map<byte[],NavigableSet<byte[]>> |
getFamilyMap()
Getting the familyMap n
|
Filter |
getFilter()
n
|
Map<String,Object> |
getFingerprint()
Compile the table and column family (i.e.
|
int |
getLimit()
Returns the limit of rows for this scan
|
long |
getMaxResultSize()
Returns the maximum result size in bytes.
|
int |
getMaxResultsPerColumnFamily()
Returns maximum number of values to return per row per CF
|
int |
getMaxVersions()
Returns the max number of versions to fetch
|
(package private) long |
getMvccReadPoint()
Get the mvcc read point used to open a scanner.
|
Scan.ReadType |
getReadType()
Returns the read type for this scan
|
int |
getRowOffsetPerColumnFamily()
Method for retrieving the scan's offset per row per column family (#kvs to be skipped)
|
ScanMetrics |
getScanMetrics()
Deprecated.
Use
ResultScanner.getScanMetrics() instead. And notice that, please do not
use this method and ResultScanner.getScanMetrics() together, the metrics
will be messed up. |
byte[] |
getStartRow()
Returns the startrow
|
byte[] |
getStopRow()
Returns the stoprow
|
TimeRange |
getTimeRange()
n
|
boolean |
hasFamilies()
Returns true if familyMap is non empty, false otherwise
|
boolean |
hasFilter()
Returns true is a filter has been specified, false if not
|
boolean |
includeStartRow()
Returns if we should include start row when scan
|
boolean |
includeStopRow()
Returns if we should include stop row when scan
|
Boolean |
isAsyncPrefetch() |
boolean |
isGetScan() |
boolean |
isNeedCursorResult() |
boolean |
isRaw()
Returns True if this Scan is in "raw" mode.
|
boolean |
isReversed()
Get whether this scan is a reversed one.
|
boolean |
isScanMetricsEnabled()
Returns True if collection of scan metrics is enabled.
|
boolean |
isSmall()
Deprecated.
since 2.0.0 and will be removed in 3.0.0. See the comment of
setSmall(boolean) |
int |
numFamilies()
Returns the number of families in familyMap
|
Scan |
readAllVersions()
Get all available versions.
|
Scan |
readVersions(int versions)
Get up to the specified number of versions of each column.
|
(package private) Scan |
resetMvccReadPoint()
Set the mvcc read point to -1 which means do not use it.
|
Scan |
setACL(Map<String,Permission> perms)
Set the ACL for the operation.
|
Scan |
setACL(String user,
Permission perms)
Set the ACL for the operation.
|
Scan |
setAllowPartialResults(boolean allowPartialResults)
Setting whether the caller wants to see the partial results when server returns
less-than-expected cells.
|
Scan |
setAsyncPrefetch(boolean asyncPrefetch) |
Scan |
setAttribute(String name,
byte[] value)
Sets an attribute.
|
Scan |
setAuthorizations(Authorizations authorizations)
Sets the authorizations to be used by this Query n
|
Scan |
setBatch(int batch)
Set the maximum number of cells to return for each call to next().
|
Scan |
setCacheBlocks(boolean cacheBlocks)
Set whether blocks should be cached for this Scan.
|
Scan |
setCaching(int caching)
Set the number of rows for caching that will be passed to scanners.
|
Scan |
setColumnFamilyTimeRange(byte[] cf,
long minStamp,
long maxStamp)
Get versions of columns only within the specified timestamp range, [minStamp, maxStamp) on a
per CF bases.
|
Scan |
setConsistency(Consistency consistency)
Sets the consistency level for this operation
|
Scan |
setFamilyMap(Map<byte[],NavigableSet<byte[]>> familyMap)
Setting the familyMap
|
Scan |
setFilter(Filter filter)
Apply the specified server-side filter when performing the Query.
|
Scan |
setId(String id)
This method allows you to set an identifier on an operation.
|
Scan |
setIsolationLevel(IsolationLevel level)
Set the isolation level for this query.
|
Scan |
setLimit(int limit)
Set the limit of rows for this scan.
|
Scan |
setLoadColumnFamiliesOnDemand(boolean value)
Set the value indicating whether loading CFs on demand should be allowed (cluster default is
false).
|
Scan |
setMaxResultSize(long maxResultSize)
Set the maximum result size.
|
Scan |
setMaxResultsPerColumnFamily(int limit)
Set the maximum number of values to return per row per Column Family
|
Scan |
setMaxVersions()
Deprecated.
|
Scan |
setMaxVersions(int maxVersions)
Deprecated.
|
(package private) Scan |
setMvccReadPoint(long mvccReadPoint)
Set the mvcc read point used to open a scanner.
|
Scan |
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.
|
Scan |
setOneRowLimit()
Call this when you only want to get one row.
|
Scan |
setPriority(int priority) |
Scan |
setRaw(boolean raw)
Enable/disable "raw" mode for this scan.
|
Scan |
setReadType(Scan.ReadType readType)
Set the read type for this scan.
|
Scan |
setReplicaId(int Id)
Specify region replica id where Query will fetch data from.
|
Scan |
setReversed(boolean reversed)
Set whether this scan is a reversed one
|
Scan |
setRowOffsetPerColumnFamily(int offset)
Set offset for the row per Column Family.
|
Scan |
setRowPrefixFilter(byte[] rowPrefix)
Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey
starts with the specified prefix.
|
Scan |
setScanMetricsEnabled(boolean enabled)
Enable collection of
ScanMetrics . |
Scan |
setSmall(boolean small)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
setLimit(int) and
setReadType(ReadType) instead. And for the one rpc optimization, now we
will also fetch data when openScanner, and if the number of rows reaches the limit
then we will close the scanner automatically which means we will fall back to one
rpc. |
Scan |
setStartRow(byte[] startRow)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
withStartRow(byte[])
instead. This method may change the inclusive of the stop row to keep compatible
with the old behavior. |
Scan |
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.
|
Scan |
setStopRow(byte[] stopRow)
Deprecated.
since 2.0.0 and will be removed in 3.0.0. Use
withStopRow(byte[]) instead.
This method may change the inclusive of the stop row to keep compatible with the
old behavior. |
Scan |
setTimeRange(long minStamp,
long maxStamp)
Get versions of columns only within the specified timestamp range, [minStamp, maxStamp).
|
Scan |
setTimestamp(long timestamp)
Get versions of columns with the specified timestamp.
|
Scan |
setTimeStamp(long timestamp)
Deprecated.
|
Map<String,Object> |
toMap(int maxCols)
Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a
Map along with the fingerprinted information.
|
Scan |
withStartRow(byte[] startRow)
Set the start row of the scan.
|
Scan |
withStartRow(byte[] startRow,
boolean inclusive)
Set the start row of the scan.
|
Scan |
withStopRow(byte[] stopRow)
Set the stop row of the scan.
|
Scan |
withStopRow(byte[] stopRow,
boolean inclusive)
Set the stop row of the scan.
|
doLoadColumnFamiliesOnDemand, getACL, getAuthorizations, getColumnFamilyTimeRange, getConsistency, getIsolationLevel, getLoadColumnFamiliesOnDemandValue, getReplicaId
getAttribute, getAttributeSize, getAttributesMap, getId, getPriority
private static final org.slf4j.Logger LOG
private static final String RAW_ATTR
private byte[] startRow
private boolean includeStartRow
private byte[] stopRow
private boolean includeStopRow
private int maxVersions
private int batch
private boolean allowPartialResults
Result
s are Result
s must be combined to form a complete Result
.
The Result
s 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.private int storeLimit
private int storeOffset
@Deprecated public static final String SCAN_ATTRIBUTES_METRICS_ENABLE
setScanMetricsEnabled(boolean)
@Deprecated public static final String SCAN_ATTRIBUTES_METRICS_DATA
getScanMetrics()
public static final String SCAN_ATTRIBUTES_TABLE_NAME
private int caching
HConstants.HBASE_CLIENT_SCANNER_CACHING
(default to HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING
) will be usedprivate long maxResultSize
private boolean cacheBlocks
private boolean reversed
private Map<byte[],NavigableSet<byte[]>> familyMap
private Boolean asyncPrefetch
public static final String HBASE_CLIENT_SCANNER_ASYNC_PREFETCH
public static final boolean DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH
HBASE_CLIENT_SCANNER_ASYNC_PREFETCH
.private boolean small
private long mvccReadPoint
private int limit
private Scan.ReadType readType
private boolean needCursorResult
public Scan()
@Deprecated public Scan(byte[] startRow, Filter filter)
new Scan().withStartRow(startRow).setFilter(filter)
instead.@Deprecated public Scan(byte[] startRow)
new Scan().withStartRow(startRow)
instead.If the specified row does not exist, the Scanner will start from the next closest row after the specified row.
startRow
- row to start scanner at or after@Deprecated public Scan(byte[] startRow, byte[] stopRow)
new Scan().withStartRow(startRow).withStopRow(stopRow)
instead.startRow
- row to start scanner at or after (inclusive)stopRow
- row to stop scanner before (exclusive)public Scan(Scan scan) throws IOException
scan
- The scan instance to copy from.IOException
- When copying the values fails.public boolean isGetScan()
public Scan addFamily(byte[] family)
Overrides previous calls to addColumn for this family.
family
- family name npublic Scan addColumn(byte[] family, byte[] qualifier)
Overrides previous calls to addFamily for this family.
family
- family namequalifier
- column qualifier npublic Scan setTimeRange(long minStamp, long maxStamp) throws IOException
minStamp
- minimum timestamp value, inclusivemaxStamp
- maximum timestamp value, exclusiveIOException
setMaxVersions()
,
n
@Deprecated public Scan setTimeStamp(long timestamp) throws IOException
timestamp
- version timestampIOException
setMaxVersions()
,
n * @deprecated As of release 2.0.0, this will be removed in HBase
3.0.0. Use {@link #setTimestamp(long)} instead
public Scan setTimestamp(long timestamp)
timestamp
- version timestampsetMaxVersions()
,
n
public Scan setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp)
Query
setColumnFamilyTimeRange
in class Query
cf
- the column family for which you want to restrictminStamp
- minimum timestamp value, inclusivemaxStamp
- maximum timestamp value, exclusive n@Deprecated public Scan setStartRow(byte[] startRow)
withStartRow(byte[])
instead. This method may change the inclusive of the stop row to keep compatible
with the old behavior.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[])
or
setStartStopRowForPrefixScan(byte[])
. Doing so will make the scan result
unexpected or even undefined.
startRow
- row to start scanner at or after n * @throws IllegalArgumentException if
startRow does not meet criteria for a row key (when length exceeds
HConstants.MAX_ROW_LENGTH
)withStartRow(byte[])
,
HBASE-17320public Scan withStartRow(byte[] startRow)
If the specified row does not exist, the Scanner will start from the next closest row after the specified row.
startRow
- row to start scanner at or after n * @throws IllegalArgumentException if
startRow does not meet criteria for a row key (when length exceeds
HConstants.MAX_ROW_LENGTH
)public Scan withStartRow(byte[] startRow, boolean inclusive)
If the specified row does not exist, or the inclusive
is false
, the Scanner
will start from the next closest row after the specified row.
Note: Do NOT use this in combination with setRowPrefixFilter(byte[])
or
setStartStopRowForPrefixScan(byte[])
. Doing so will make the scan result
unexpected or even undefined.
startRow
- row to start scanner at or afterinclusive
- whether we should include the start row when scan n * @throws
IllegalArgumentException if startRow does not meet criteria for a row key
(when length exceeds HConstants.MAX_ROW_LENGTH
)@Deprecated public Scan setStopRow(byte[] stopRow)
withStopRow(byte[])
instead.
This method may change the inclusive of the stop row to keep compatible with the
old behavior.The scan will include rows that are lexicographically less than the provided stopRow.
Note: Do NOT use this in combination with setRowPrefixFilter(byte[])
or
setStartStopRowForPrefixScan(byte[])
. Doing so will make the scan result
unexpected or even undefined.
stopRow
- row to end at (exclusive) n * @throws IllegalArgumentException if stopRow does
not meet criteria for a row key (when length exceeds
HConstants.MAX_ROW_LENGTH
)withStopRow(byte[])
,
HBASE-17320public Scan withStopRow(byte[] stopRow)
The scan will include rows that are lexicographically less than the provided stopRow.
Note: When doing a filter for a rowKey Prefix use
setRowPrefixFilter(byte[])
. The 'trailing 0' will not yield the desired result.
stopRow
- row to end at (exclusive) n * @throws IllegalArgumentException if stopRow does
not meet criteria for a row key (when length exceeds
HConstants.MAX_ROW_LENGTH
)public Scan withStopRow(byte[] stopRow, boolean inclusive)
The scan will include rows that are lexicographically less than (or equal to if
inclusive
is true
) the provided stopRow.
Note: Do NOT use this in combination with setRowPrefixFilter(byte[])
or
setStartStopRowForPrefixScan(byte[])
. Doing so will make the scan result
unexpected or even undefined.
stopRow
- row to end atinclusive
- whether we should include the stop row when scan n * @throws
IllegalArgumentException if stopRow does not meet criteria for a row key (when
length exceeds HConstants.MAX_ROW_LENGTH
)public Scan setRowPrefixFilter(byte[] rowPrefix)
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.
rowPrefix
- the prefix all rows must start with. (Set null to remove the filter.) n
* @deprecated 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 a Filter
but uses
setting the startRow and stopRow instead. Use
setStartStopRowForPrefixScan(byte[])
instead.public Scan 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.
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.
rowPrefix
- the prefix all rows must start with. (Set null to remove the filter.) n@Deprecated public Scan setMaxVersions()
readAllVersions()
instead.readAllVersions()
,
HBASE-17125@Deprecated public Scan setMaxVersions(int maxVersions)
maxVersions
- maximum versions for each column n * @deprecated since 2.0.0 and will be
removed in 3.0.0. It is easy to misunderstand with column family's max
versions, so use readVersions(int)
instead.readVersions(int)
,
HBASE-17125public Scan readAllVersions()
public Scan readVersions(int versions)
versions
- specified number of versions for each column npublic Scan setBatch(int batch)
setAllowPartialResults(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.batch
- the maximum number of valuesResult.mayHaveMoreCellsInRow()
public Scan setMaxResultsPerColumnFamily(int limit)
limit
- the maximum number of values returned / row / CFpublic Scan setRowOffsetPerColumnFamily(int offset)
offset
- is the number of kvs that will be skipped.public Scan setCaching(int caching)
HConstants.HBASE_CLIENT_SCANNER_CACHING
will apply. Higher
caching values will enable faster scanners but will use more memory.caching
- the number of rows for cachingpublic long getMaxResultSize()
setMaxResultSize(long)
public Scan setMaxResultSize(long maxResultSize)
maxResultSize
- The maximum result size in bytes.public Scan setFilter(Filter filter)
Query
Filter.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.public Scan setFamilyMap(Map<byte[],NavigableSet<byte[]>> familyMap)
familyMap
- map of family to qualifier npublic Map<byte[],NavigableSet<byte[]>> getFamilyMap()
public int numFamilies()
public boolean hasFamilies()
public byte[][] getFamilies()
public byte[] getStartRow()
public boolean includeStartRow()
public byte[] getStopRow()
public boolean includeStopRow()
public int getMaxVersions()
public int getBatch()
public int getMaxResultsPerColumnFamily()
public int getRowOffsetPerColumnFamily()
public int getCaching()
public TimeRange getTimeRange()
public boolean hasFilter()
public Scan setCacheBlocks(boolean cacheBlocks)
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).
cacheBlocks
- if false, default settings are overridden and blocks will not be cachedpublic boolean getCacheBlocks()
public Scan setReversed(boolean reversed)
This is false by default which means forward(normal) scan.
reversed
- if true, scan will be backward order npublic boolean isReversed()
public Scan setAllowPartialResults(boolean allowPartialResults)
setBatch(int)
public boolean getAllowPartialResults()
ResultScanner.next()
public Scan setLoadColumnFamiliesOnDemand(boolean value)
Query
setLoadColumnFamiliesOnDemand
in class Query
public Map<String,Object> getFingerprint()
getFingerprint
in class Operation
public Map<String,Object> toMap(int maxCols)
public Scan setRaw(boolean raw)
raw
- True/False to enable/disable "raw" mode.public boolean isRaw()
@Deprecated public Scan setSmall(boolean small)
setLimit(int)
and
setReadType(ReadType)
instead. And for the one rpc optimization, now we
will also fetch data when openScanner, and if the number of rows reaches the limit
then we will close the scanner automatically which means we will fall back to one
rpc.Small scan should use pread and big scan can use seek + read seek + read is fast but can cause two problem (1) resource contention (2) cause too much network io [89-fb] Using pread for non-compaction read request https://issues.apache.org/jira/browse/HBASE-7266 On the other hand, if setting it true, we would do openScanner,next,closeScanner in one RPC call. It means the better performance for small scan. [HBASE-9488]. Generally, if the scan range is within one data block(64KB), it could be considered as a small scan.
small
- set if that should use read type of PREADsetLimit(int)
,
setReadType(ReadType)
,
HBASE-17045@Deprecated public boolean isSmall()
setSmall(boolean)
public Scan setAttribute(String name, byte[] value)
Attributes
setAttribute
in interface Attributes
setAttribute
in class OperationWithAttributes
name
- attribute namevalue
- attribute valuepublic Scan setId(String id)
OperationWithAttributes
setId
in class OperationWithAttributes
public Scan setAuthorizations(Authorizations authorizations)
Query
setAuthorizations
in class Query
public Scan setACL(Map<String,Permission> perms)
Query
public Scan setACL(String user, Permission perms)
Query
public Scan setConsistency(Consistency consistency)
Query
setConsistency
in class Query
consistency
- the consistency levelpublic Scan setReplicaId(int Id)
Query
Query.setConsistency(Consistency)
passing Consistency.TIMELINE
to read data from a
specific replicaId. setReplicaId
in class Query
public Scan setIsolationLevel(IsolationLevel level)
Query
setIsolationLevel
in class Query
level
- IsolationLevel for this querypublic Scan setPriority(int priority)
setPriority
in class OperationWithAttributes
public Scan setScanMetricsEnabled(boolean enabled)
ScanMetrics
. For advanced users.enabled
- Set to true to enable accumulating scan metricspublic boolean isScanMetricsEnabled()
@Deprecated public ScanMetrics getScanMetrics()
ResultScanner.getScanMetrics()
instead. And notice that, please do not
use this method and ResultScanner.getScanMetrics()
together, the metrics
will be messed up.setScanMetricsEnabled(boolean)
public Boolean isAsyncPrefetch()
public Scan setAsyncPrefetch(boolean asyncPrefetch)
public int getLimit()
public Scan setLimit(int limit)
This condition will be tested at last, after all other conditions such as stopRow, filter, etc.
limit
- the limit of rows for this scan npublic Scan setOneRowLimit()
limit
to 1
, and also
set readType
to Scan.ReadType.PREAD
. npublic Scan.ReadType getReadType()
public Scan setReadType(Scan.ReadType readType)
Notice that we may choose to use pread even if you specific Scan.ReadType.STREAM
here. For
example, we will always use pread if this is a get scan. n
long getMvccReadPoint()
Scan setMvccReadPoint(long mvccReadPoint)
Scan resetMvccReadPoint()
public Scan setNeedCursorResult(boolean needCursorResult)
Result.isCursor()
Result.getCursor()
Cursor
public boolean isNeedCursorResult()
public static Scan createScanFromCursor(Cursor cursor)
Result.isCursor()
Result.getCursor()
Cursor
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.