Interface Region
- All Superinterfaces:
ConfigurationObserver
- All Known Implementing Classes:
HRegion
Coprocessors
. The operations include ability to do mutations, requesting compaction, getting
different counters/sizes, locking rows and getting access to Stores.-
Nested Class Summary
Modifier and TypeInterfaceDescriptionstatic enum
Operation enum is used instartRegionOperation()
and elsewhere to provide context for various checks.static interface
Row lock held by a given thread. -
Method Summary
Modifier and TypeMethodDescriptionPerform one or more append operations on a row.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 3.0.0 and will be removed in 4.0.0.boolean
checkAndMutate
(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
checkAndMutate
(byte[] row, Filter filter, Mutation mutation) Deprecated.since 3.0.0 and will be removed in 4.0.0.boolean
checkAndMutate
(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) Deprecated.since 3.0.0 and will be removed in 4.0.0.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 3.0.0 and will be removed in 4.0.0.boolean
checkAndRowMutate
(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, RowMutations mutations) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
checkAndRowMutate
(byte[] row, Filter filter, RowMutations mutations) Deprecated.since 3.0.0 and will be removed in 4.0.0.boolean
checkAndRowMutate
(byte[] row, Filter filter, TimeRange timeRange, RowMutations mutations) Deprecated.since 3.0.0 and will be removed in 4.0.0.void
Closes the region operation lock.void
Closes the region operation lock.void
Deletes the specified cells/row.Do a get based on the get parameter.Do a get based on the get parameter.long
Returns the number of blocked requestsThe comparator to be used with the regionlong
Returns the number of failed checkAndMutate guardslong
Returns the number of checkAndMutate guards that passedReturns if a given region is in compaction now.long
Returns coprocessor requests count for this regionlong
Returns the size of data processed bypassing the WAL, in byteslong
long
Returns filtered read requests count for this regionlong
long
long
long
int
The minimum block size configuration from all relevant column families.long
Returns the number of mutations processed bypassing the WALlong
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
long
Returns read requests count for this regionReturns region information for this regiongetRowLock
(byte[] row, boolean readLock) Get a row lock for the specified row.getScanner
(Scan scan) Return an iterator that scans over the HRegion, returning the indicated columns and rows specified by theScan
.getScanner
(Scan scan, List<KeyValueScanner> additionalScanners) Return an iterator that scans over the HRegion, returning the indicated columns and rows specified by theScan
.getStore
(byte[] family) Return the Store for the given familygetStoreFileList
(byte[][] columns) Returns list of store file names for the given familiesReturn the list of Stores managed by this regionReturns table descriptor for this regionlong
Returns write request count for this regionPerform one or more increment operations on a row.boolean
Returns true if region is available (not closed and not closing)boolean
isClosed()
Returns true if region is closedboolean
Returns True if closing process has startedboolean
Returns true if region is mergeableboolean
Returns True if region is read onlyboolean
Returns true if region is splittablemutateRow
(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
Puts some data in the table.boolean
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 familyvoid
requestCompaction
(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) Request compaction on this region.void
requestFlush
(FlushLifeCycleTracker tracker) Request flush on this region.void
This method needs to be called before any public call that reads or modifies data.void
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 completeMethods inherited from interface org.apache.hadoop.hbase.conf.ConfigurationObserver
onConfigurationChange
-
Method Details
-
getRegionInfo
Returns region information for this region -
getTableDescriptor
Returns table descriptor for this region -
isAvailable
boolean isAvailable()Returns true if region is available (not closed and not closing) -
isClosed
boolean isClosed()Returns true if region is closed -
isClosing
boolean isClosing()Returns True if closing process has started -
isReadOnly
boolean isReadOnly()Returns True if region is read only -
isSplittable
boolean isSplittable()Returns true if region is splittable -
isMergeable
boolean isMergeable()Returns true if region is mergeable -
getStores
Return the list of Stores managed by this regionUse with caution. Exposed for use of fixup utilities.
- Returns:
- a list of the Stores managed by this region
-
getStore
Return the Store for the given familyUse with caution. Exposed for use of fixup utilities.
- Returns:
- the Store for the given family
-
getStoreFileList
Returns list of store file names for the given families -
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.- Throws:
IOException
-
getMaxFlushedSeqId
long getMaxFlushedSeqId()- Returns:
- the max sequence id of flushed data on this region; no edit in memory will have a sequence id that is less that what is returned here.
-
getOldestHfileTs
This can be used to determine the last time all files of this region were major compacted.- Parameters:
majorCompactionOnly
- Only consider HFile that are the result of major compaction- Returns:
- the timestamp of the oldest HFile for all stores of this region
- Throws:
IOException
-
getMaxStoreSeqId
Map<byte[],Long> getMaxStoreSeqId()- Returns:
- map of column family names to max sequence id that was read from storage when this region was opened
-
getEarliestFlushTimeForAllStores
- Returns:
- The earliest time a store in the region was flushed. All other stores in the region would have been flushed either at, or after this time.
-
getReadRequestsCount
long getReadRequestsCount()Returns read requests count for this region -
getCpRequestsCount
long getCpRequestsCount()Returns coprocessor requests count for this region -
getFilteredReadRequestsCount
long getFilteredReadRequestsCount()Returns filtered read requests count for this region -
getWriteRequestsCount
long getWriteRequestsCount()Returns write request count for this region -
getMemStoreDataSize
long getMemStoreDataSize()- Returns:
- memstore size for this region, in bytes. It just accounts data size of cells added to the memstores of this Region. Means size in bytes for key, value and tags within Cells. It wont consider any java heap overhead for the cell objects or any other.
-
getMemStoreHeapSize
long getMemStoreHeapSize()- Returns:
- memstore heap size for this region, in bytes. It accounts data size of cells added to the memstores of this Region, as well as java heap overhead for the cell objects or any other.
-
getMemStoreOffHeapSize
long getMemStoreOffHeapSize()- Returns:
- memstore off-heap size for this region, in bytes. It accounts data size of cells added to the memstores of this Region, as well as overhead for the cell objects or any other that is allocated off-heap.
-
getNumMutationsWithoutWAL
long getNumMutationsWithoutWAL()Returns the number of mutations processed bypassing the WAL -
getDataInMemoryWithoutWAL
long getDataInMemoryWithoutWAL()Returns the size of data processed bypassing the WAL, in bytes -
getBlockedRequestsCount
long getBlockedRequestsCount()Returns the number of blocked requests -
getCheckAndMutateChecksPassed
Returns the number of checkAndMutate guards that passed -
getCheckAndMutateChecksFailed
Returns the number of failed checkAndMutate guards -
startRegionOperation
This method needs to be called before any public call that reads or modifies data. Acquires a read lock and checks if the region is closing or closed.closeRegionOperation()
MUST then always be called after the operation has completed, whether it succeeded or failed.- Throws:
IOException
-
startRegionOperation
This method needs to be called before any public call that reads or modifies data. Acquires a read lock and checks if the region is closing or closed.closeRegionOperation()
MUST then always be called after the operation has completed, whether it succeeded or failed.- Parameters:
op
- The operation is about to be taken on the region- Throws:
IOException
-
closeRegionOperation
Closes the region operation lock.- Throws:
IOException
-
closeRegionOperation
Closes the region operation lock. This needs to be called in the finally block corresponding to the try block ofstartRegionOperation(Operation)
- Throws:
IOException
-
getRowLock
Get a row lock for the specified row. All locks are reentrant. Before calling this function make sure that a region operation has already been started (the calling thread has already acquired the region-close-guard lock).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.
- Parameters:
row
- The row actions will be performed againstreadLock
- is the lock reader or writer. True indicates that a non-exclusive lock is requested- Throws:
IOException
- See Also:
-
append
Perform one or more append operations on a row.- Returns:
- result of the operation
- Throws:
IOException
-
batchMutate
Perform a batch of mutations.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.
- Parameters:
mutations
- the list of mutations- Returns:
- an array of OperationStatus which internally contains the OperationStatusCode and the exceptionMessage if any.
- Throws:
IOException
-
checkAndMutate
@Deprecated default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Mutation mutation) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row/family/qualifier value matches the expected value and if it does, it performs the mutation. If the passed value is null, the lack of column value (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a single row.- Parameters:
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutation
- data to put if check succeeds- Returns:
- true if mutation was applied, false otherwise
- Throws:
IOException
-
checkAndMutate
@Deprecated boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row/family/qualifier value matches the expected value and if it does, it performs the mutation. If the passed value is null, the lack of column value (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a single row.- Parameters:
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 check- Returns:
- true if mutation was applied, false otherwise
- Throws:
IOException
-
checkAndMutate
@Deprecated default boolean checkAndMutate(byte[] row, Filter filter, Mutation mutation) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row matches the filter and if it does, it performs the mutation. See checkAndRowMutate to do many checkAndPuts at a time on a single row.- Parameters:
row
- to checkfilter
- the filtermutation
- data to put if check succeeds- Returns:
- true if mutation was applied, false otherwise
- Throws:
IOException
-
checkAndMutate
@Deprecated boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row value matches the filter and if it does, it performs the mutation. See checkAndRowMutate to do many checkAndPuts at a time on a single row.- Parameters:
row
- to checkfilter
- the filtermutation
- data to put if check succeedstimeRange
- time range to check- Returns:
- true if mutation was applied, false otherwise
- Throws:
IOException
-
checkAndRowMutate
@Deprecated default boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, RowMutations mutations) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row/family/qualifier value matches the expected values and if it does, it performs the row mutations. If the passed value is null, the lack of column value (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate to do one checkAndMutate at a time.- Parameters:
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorcomparator
- the expected valuemutations
- data to put if check succeeds- Returns:
- true if mutations were applied, false otherwise
- Throws:
IOException
-
checkAndRowMutate
@Deprecated boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, RowMutations mutations) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row/family/qualifier value matches the expected values and if it does, it performs the row mutations. If the passed value is null, the lack of column value (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate to do one checkAndMutate at a time.- Parameters:
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 check- Returns:
- true if mutations were applied, false otherwise
- Throws:
IOException
-
checkAndRowMutate
@Deprecated default boolean checkAndRowMutate(byte[] row, Filter filter, RowMutations mutations) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row matches the filter and if it does, it performs the row mutations. Use to do many mutations on a single row. Use checkAndMutate to do one checkAndMutate at a time.- Parameters:
row
- to checkfilter
- the filtermutations
- data to put if check succeeds- Returns:
- true if mutations were applied, false otherwise
- Throws:
IOException
-
checkAndRowMutate
@Deprecated boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations mutations) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsecheckAndMutate(CheckAndMutate)
instead.Atomically checks if a row matches the filter and if it does, it performs the row mutations. Use to do many mutations on a single row. Use checkAndMutate to do one checkAndMutate at a time.- Parameters:
row
- to checkfilter
- the filtermutations
- data to put if check succeedstimeRange
- time range to check- Returns:
- true if mutations were applied, false otherwise
- Throws:
IOException
-
checkAndMutate
Atomically checks if a row matches the conditions and if it does, it performs the actions. Use to do many mutations on a single row. Use checkAndMutate to do one checkAndMutate at a time.- Parameters:
checkAndMutate
- the CheckAndMutate object- Returns:
- true if mutations were applied, false otherwise
- Throws:
IOException
- if an error occurred in this method
-
delete
Deletes the specified cells/row.- Throws:
IOException
-
get
Do a get based on the get parameter.- Parameters:
get
- query parameters- Returns:
- result of the operation
- Throws:
IOException
-
get
Do a get based on the get parameter.- Parameters:
get
- query parameterswithCoprocessor
- invoke coprocessor or not. We don't want to always invoke cp.- Returns:
- list of cells resulting from the operation
- Throws:
IOException
-
getScanner
Return an iterator that scans over the HRegion, returning the indicated columns and rows specified by theScan
.This Iterator must be closed by the caller.
- Parameters:
scan
- configuredScan
- Throws:
IOException
- read exceptions
-
getScanner
Return an iterator that scans over the HRegion, returning the indicated columns and rows specified by theScan
. 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 RegionThis Iterator must be closed by the caller.
- Parameters:
scan
- configuredScan
additionalScanners
- Any additional scanners to be used- Throws:
IOException
- read exceptions
-
getCellComparator
The comparator to be used with the region -
increment
Perform one or more increment operations on a row.- Returns:
- result of the operation
- Throws:
IOException
-
mutateRow
Performs multiple mutations atomically on a single row.- Parameters:
mutations
- object that specifies the set of mutations to perform atomically- Returns:
- results of Increment/Append operations. If no Increment/Append operations, it returns null
- Throws:
IOException
-
mutateRowsWithLocks
void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException Perform atomic mutations within the region.- Parameters:
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 thatrowsToLock
is sorted in order to avoid deadlocks.- Throws:
IOException
-
put
Puts some data in the table.- Throws:
IOException
-
getCompactionState
Returns if a given region is in compaction now. -
requestCompaction
void requestCompaction(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException Request compaction on this region.- Throws:
IOException
-
requestCompaction
void requestCompaction(byte[] family, String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException Request compaction for the given family- Throws:
IOException
-
requestFlush
Request flush on this region.- Throws:
IOException
-
waitForFlushes
Wait for all current flushes of the region to complete- Parameters:
timeout
- The maximum time to wait in milliseconds.- Returns:
- False when timeout elapsed but flushes are not over. True when flushes are over within max wait time period.
-
getReadOnlyConfiguration
org.apache.hadoop.conf.Configuration getReadOnlyConfiguration()- Returns:
- a read only configuration of this region; throws
UnsupportedOperationException
if you try to set a configuration.
-
getMinBlockSizeBytes
int getMinBlockSizeBytes()The minimum block size configuration from all relevant column families. This is used when estimating quota consumption.
-