@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface Region extends ConfigurationObserver
Coprocessors
. The operations include ability to do mutations, requesting compaction, getting
different counters/sizes, locking rows and getting access to Stores.Modifier and Type | Interface and Description |
---|---|
static class |
Region.Operation
Operation enum is used in
startRegionOperation() and elsewhere to provide context
for various checks. |
static interface |
Region.RowLock
Row lock held by a given thread.
|
Modifier and Type | Method and Description |
---|---|
Result |
append(Append append)
Perform one or more append operations on a row.
|
OperationStatus[] |
batchMutate(Mutation[] mutations)
Perform a batch of mutations.
|
default boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Mutation mutation)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
TimeRange timeRange,
Mutation mutation)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
default boolean |
checkAndMutate(byte[] row,
Filter filter,
Mutation mutation)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
boolean |
checkAndMutate(byte[] row,
Filter filter,
TimeRange timeRange,
Mutation mutation)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
CheckAndMutateResult |
checkAndMutate(CheckAndMutate checkAndMutate)
Atomically checks if a row matches the conditions and if it does, it performs the actions.
|
default boolean |
checkAndRowMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
RowMutations mutations)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
boolean |
checkAndRowMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
TimeRange timeRange,
RowMutations mutations)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
default boolean |
checkAndRowMutate(byte[] row,
Filter filter,
RowMutations mutations)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
boolean |
checkAndRowMutate(byte[] row,
Filter filter,
TimeRange timeRange,
RowMutations mutations)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
checkAndMutate(CheckAndMutate) instead. |
void |
closeRegionOperation()
Closes the region operation lock.
|
void |
closeRegionOperation(Region.Operation op)
Closes the region operation lock.
|
void |
delete(Delete delete)
Deletes the specified cells/row.
|
Result |
get(Get get)
Do a get based on the get parameter.
|
List<Cell> |
get(Get get,
boolean withCoprocessor)
Do a get based on the get parameter.
|
long |
getBlockedRequestsCount()
Returns the number of blocked requests
|
CellComparator |
getCellComparator()
The comparator to be used with the region
|
long |
getCheckAndMutateChecksFailed()
Returns the number of failed checkAndMutate guards
|
long |
getCheckAndMutateChecksPassed()
Returns the number of checkAndMutate guards that passed
|
CompactionState |
getCompactionState()
Returns if a given region is in compaction now.
|
long |
getDataInMemoryWithoutWAL()
Returns the size of data processed bypassing the WAL, in bytes
|
long |
getEarliestFlushTimeForAllStores() |
long |
getFilteredReadRequestsCount()
Returns filtered read requests count for this region
|
long |
getMaxFlushedSeqId() |
Map<byte[],Long> |
getMaxStoreSeqId() |
long |
getMemStoreDataSize() |
long |
getMemStoreHeapSize() |
long |
getMemStoreOffHeapSize() |
long |
getNumMutationsWithoutWAL()
Returns the number of mutations processed bypassing the WAL
|
long |
getOldestHfileTs(boolean majorCompactionOnly)
This can be used to determine the last time all files of this region were major compacted.
|
org.apache.hadoop.conf.Configuration |
getReadOnlyConfiguration() |
long |
getReadRequestsCount()
Returns read requests count for this region
|
RegionInfo |
getRegionInfo()
Returns region information for this region
|
Region.RowLock |
getRowLock(byte[] row,
boolean readLock)
Get a row lock for the specified row.
|
RegionScanner |
getScanner(Scan scan)
Return an iterator that scans over the HRegion, returning the indicated columns and rows
specified by the
Scan . |
RegionScanner |
getScanner(Scan scan,
List<KeyValueScanner> additionalScanners)
Return an iterator that scans over the HRegion, returning the indicated columns and rows
specified by the
Scan . |
Store |
getStore(byte[] family)
Return the Store for the given family
|
List<String> |
getStoreFileList(byte[][] columns)
Returns list of store file names for the given families
|
List<? extends Store> |
getStores()
Return the list of Stores managed by this region
|
TableDescriptor |
getTableDescriptor()
Returns table descriptor for this region
|
long |
getWriteRequestsCount()
Returns write request count for this region
|
Result |
increment(Increment increment)
Perform one or more increment operations on a row.
|
boolean |
isAvailable()
Returns true if region is available (not closed and not closing)
|
boolean |
isClosed()
Returns true if region is closed
|
boolean |
isClosing()
Returns True if closing process has started
|
boolean |
isMergeable()
Returns true if region is mergeable
|
boolean |
isReadOnly()
Returns True if region is read only
|
boolean |
isSplittable()
Returns true if region is splittable
|
Result |
mutateRow(RowMutations mutations)
Performs multiple mutations atomically on a single row.
|
void |
mutateRowsWithLocks(Collection<Mutation> mutations,
Collection<byte[]> rowsToLock,
long nonceGroup,
long nonce)
Perform atomic mutations within the region.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use
Coprocessors instead.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor,
long nonceGroup,
long nonce)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use
Coprocessors instead.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor,
long timeout,
long nonceGroup,
long nonce)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use
Coprocessors instead.
|
void |
put(Put put)
Puts some data in the table.
|
boolean |
refreshStoreFiles()
Check the region's underlying store files, open the files that have not been opened yet, and
remove the store file readers for store files no longer available.
|
void |
requestCompaction(byte[] family,
String why,
int priority,
boolean major,
CompactionLifeCycleTracker tracker)
Request compaction for the given family
|
void |
requestCompaction(String why,
int priority,
boolean major,
CompactionLifeCycleTracker tracker)
Request compaction on this region.
|
void |
requestFlush(FlushLifeCycleTracker tracker)
Request flush on this region.
|
void |
startRegionOperation()
This method needs to be called before any public call that reads or modifies data.
|
void |
startRegionOperation(Region.Operation op)
This method needs to be called before any public call that reads or modifies data.
|
boolean |
waitForFlushes(long timeout)
Wait for all current flushes of the region to complete
|
onConfigurationChange
RegionInfo getRegionInfo()
TableDescriptor getTableDescriptor()
boolean isAvailable()
boolean isClosed()
boolean isClosing()
boolean isReadOnly()
boolean isSplittable()
boolean isMergeable()
List<? extends Store> getStores()
Use with caution. Exposed for use of fixup utilities.
Store getStore(byte[] family)
Use with caution. Exposed for use of fixup utilities.
List<String> getStoreFileList(byte[][] columns)
boolean refreshStoreFiles() throws IOException
IOException
long getMaxFlushedSeqId()
long getOldestHfileTs(boolean majorCompactionOnly) throws IOException
majorCompactionOnly
- Only consider HFile that are the result of major compactionIOException
Map<byte[],Long> getMaxStoreSeqId()
long getEarliestFlushTimeForAllStores()
long getReadRequestsCount()
long getFilteredReadRequestsCount()
long getWriteRequestsCount()
long getMemStoreDataSize()
long getMemStoreHeapSize()
long getMemStoreOffHeapSize()
long getNumMutationsWithoutWAL()
long getDataInMemoryWithoutWAL()
long getBlockedRequestsCount()
long getCheckAndMutateChecksPassed()
long getCheckAndMutateChecksFailed()
void startRegionOperation() throws IOException
closeRegionOperation()
MUST then always be called after the operation has completed,
whether it succeeded or failed. n
IOException
void startRegionOperation(Region.Operation op) throws IOException
closeRegionOperation()
MUST then always be called after the operation has completed,
whether it succeeded or failed.
op
- The operation is about to be taken on the region nIOException
void closeRegionOperation() throws IOException
IOException
void closeRegionOperation(Region.Operation op) throws IOException
startRegionOperation(Operation)
nIOException
Region.RowLock getRowLock(byte[] row, boolean readLock) throws IOException
The obtained locks should be released after use by Region.RowLock.release()
NOTE: the boolean passed here has changed. It used to be a boolean that stated whether or not to wait on the lock. Now it is whether it an exclusive lock is requested.
row
- The row actions will be performed againstreadLock
- is the lock reader or writer. True indicates that a non-exclusive lock is
requestedIOException
startRegionOperation()
,
startRegionOperation(Operation)
Result append(Append append) throws IOException
IOException
OperationStatus[] batchMutate(Mutation[] mutations) throws IOException
Please do not operate on a same column of a single row in a batch, we will not consider the previous operation in the same batch when performing the operations in the batch.
mutations
- the list of mutationsIOException
@Deprecated default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Mutation mutation) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutation
- data to put if check succeedsIOException
@Deprecated boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutation
- data to put if check succeedstimeRange
- time range to checkIOException
@Deprecated default boolean checkAndMutate(byte[] row, Filter filter, Mutation mutation) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfilter
- the filtermutation
- data to put if check succeedsIOException
@Deprecated boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfilter
- the filtermutation
- data to put if check succeedstimeRange
- time range to checkIOException
@Deprecated default boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, RowMutations mutations) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutations
- data to put if check succeedsIOException
@Deprecated boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, RowMutations mutations) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutations
- data to put if check succeedstimeRange
- time range to checkIOException
@Deprecated default boolean checkAndRowMutate(byte[] row, Filter filter, RowMutations mutations) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfilter
- the filtermutations
- data to put if check succeedsIOException
@Deprecated boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations mutations) throws IOException
checkAndMutate(CheckAndMutate)
instead.row
- to checkfilter
- the filtermutations
- data to put if check succeedstimeRange
- time range to checkIOException
CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException
checkAndMutate
- the CheckAndMutate objectIOException
- if an error occurred in this methodvoid delete(Delete delete) throws IOException
IOException
Result get(Get get) throws IOException
get
- query parametersIOException
List<Cell> get(Get get, boolean withCoprocessor) throws IOException
get
- query parameterswithCoprocessor
- invoke coprocessor or not. We don't want to always invoke cp.IOException
RegionScanner getScanner(Scan scan) throws IOException
Scan
.
This Iterator must be closed by the caller.
scan
- configured Scan
n * @throws IOException read exceptionsIOException
RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException
Scan
. The scanner will also include the additional scanners passed
along with the scanners for the specified Scan instance. Should be careful with the usage to
pass additional scanners only within this Region
This Iterator must be closed by the caller.
scan
- configured Scan
additionalScanners
- Any additional scanners to be used n * @throws IOException read
exceptionsIOException
CellComparator getCellComparator()
Result increment(Increment increment) throws IOException
IOException
Result mutateRow(RowMutations mutations) throws IOException
mutations
- object that specifies the set of mutations to perform atomicallyIOException
void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException
mutations
- The list of mutations to perform. mutations
can contain
operations for multiple rows. Caller has to ensure that all rows are
contained in this region.rowsToLock
- Rows to locknonceGroup
- Optional nonce group of the operation (client Id)nonce
- Optional nonce of the operation (unique random id to ensure "more
idempotence") If multiple rows are locked care should be taken that
rowsToLock
is sorted in order to avoid deadlocks. nIOException
@Deprecated void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException
processor
- The object defines the reads and writes to a row.IOException
@Deprecated void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce) throws IOException
processor
- The object defines the reads and writes to a row.nonceGroup
- Optional nonce group of the operation (client Id)nonce
- Optional nonce of the operation (unique random id to ensure "more
idempotence")IOException
@Deprecated void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, long nonceGroup, long nonce) throws IOException
processor
- The object defines the reads and writes to a row.timeout
- The timeout of the processor.process() execution Use a negative number to
switch off the time boundnonceGroup
- Optional nonce group of the operation (client Id)nonce
- Optional nonce of the operation (unique random id to ensure "more
idempotence")IOException
void put(Put put) throws IOException
IOException
CompactionState getCompactionState()
void requestCompaction(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException
IOException
void requestCompaction(byte[] family, String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException
IOException
void requestFlush(FlushLifeCycleTracker tracker) throws IOException
IOException
boolean waitForFlushes(long timeout)
timeout
- The maximum time to wait in milliseconds.org.apache.hadoop.conf.Configuration getReadOnlyConfiguration()
UnsupportedOperationException
if you try to set a configuration.Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.