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 SummaryNested ClassesModifier and TypeInterfaceDescriptionstatic enumOperation enum is used instartRegionOperation()and elsewhere to provide context for various checks.static interfaceRow lock held by a given thread.
- 
Method SummaryModifier and TypeMethodDescriptionPerform one or more append operations on a row.batchMutate(Mutation[] mutations) Perform a batch of mutations.default booleancheckAndMutate(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.booleancheckAndMutate(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 booleancheckAndMutate(byte[] row, Filter filter, Mutation mutation) Deprecated.since 3.0.0 and will be removed in 4.0.0.booleancheckAndMutate(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 booleancheckAndRowMutate(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.booleancheckAndRowMutate(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 booleancheckAndRowMutate(byte[] row, Filter filter, RowMutations mutations) Deprecated.since 3.0.0 and will be removed in 4.0.0.booleancheckAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations mutations) Deprecated.since 3.0.0 and will be removed in 4.0.0.voidCloses the region operation lock.voidCloses the region operation lock.voidDeletes the specified cells/row.Do a get based on the get parameter.Do a get based on the get parameter.longReturns the number of blocked requestsThe comparator to be used with the regionlongReturns the number of failed checkAndMutate guardslongReturns the number of checkAndMutate guards that passedReturns if a given region is in compaction now.longReturns coprocessor requests count for this regionlongReturns the size of data processed bypassing the WAL, in byteslonglongReturns filtered read requests count for this regionlonglonglonglongintThe minimum block size configuration from all relevant column families.longReturns the number of mutations processed bypassing the WALlonggetOldestHfileTs(boolean majorCompactionOnly) This can be used to determine the last time all files of this region were major compacted.org.apache.hadoop.conf.ConfigurationlongReturns 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 regionlongReturns write request count for this regionPerform one or more increment operations on a row.booleanReturns true if region is available (not closed and not closing)booleanisClosed()Returns true if region is closedbooleanReturns True if closing process has startedbooleanReturns true if region is mergeablebooleanReturns True if region is read onlybooleanReturns true if region is splittablemutateRow(RowMutations mutations) Performs multiple mutations atomically on a single row.voidmutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, long nonceGroup, long nonce) Perform atomic mutations within the region.voidPuts some data in the table.booleanCheck 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.voidrequestCompaction(byte[] family, String why, int priority, boolean major, CompactionLifeCycleTracker tracker) Request compaction for the given familyvoidrequestCompaction(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) Request compaction on this region.voidrequestFlush(FlushLifeCycleTracker tracker) Request flush on this region.voidThis method needs to be called before any public call that reads or modifies data.voidThis method needs to be called before any public call that reads or modifies data.booleanwaitForFlushes(long timeout) Wait for all current flushes of the region to completeMethods inherited from interface org.apache.hadoop.hbase.conf.ConfigurationObserveronConfigurationChange
- 
Method Details- 
getRegionInfoReturns region information for this region
- 
getTableDescriptorReturns table descriptor for this region
- 
isAvailableboolean isAvailable()Returns true if region is available (not closed and not closing)
- 
isClosedboolean isClosed()Returns true if region is closed
- 
isClosingboolean isClosing()Returns True if closing process has started
- 
isReadOnlyboolean isReadOnly()Returns True if region is read only
- 
isSplittableboolean isSplittable()Returns true if region is splittable
- 
isMergeableboolean isMergeable()Returns true if region is mergeable
- 
getStoresReturn 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
 
- 
getStoreReturn the Store for the given familyUse with caution. Exposed for use of fixup utilities. - Returns:
- the Store for the given family
 
- 
getStoreFileListReturns list of store file names for the given families
- 
refreshStoreFilesCheck 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
 
- 
getMaxFlushedSeqIdlong 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.
 
- 
getOldestHfileTsThis 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
 
- 
getMaxStoreSeqIdMap<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.
 
- 
getReadRequestsCountlong getReadRequestsCount()Returns read requests count for this region
- 
getCpRequestsCountlong getCpRequestsCount()Returns coprocessor requests count for this region
- 
getFilteredReadRequestsCountlong getFilteredReadRequestsCount()Returns filtered read requests count for this region
- 
getWriteRequestsCountlong getWriteRequestsCount()Returns write request count for this region
- 
getMemStoreDataSizelong 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.
 
- 
getMemStoreHeapSizelong 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.
 
- 
getMemStoreOffHeapSizelong 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.
 
- 
getNumMutationsWithoutWALlong getNumMutationsWithoutWAL()Returns the number of mutations processed bypassing the WAL
- 
getDataInMemoryWithoutWALlong getDataInMemoryWithoutWAL()Returns the size of data processed bypassing the WAL, in bytes
- 
getBlockedRequestsCountlong getBlockedRequestsCount()Returns the number of blocked requests
- 
getCheckAndMutateChecksPassedReturns the number of checkAndMutate guards that passed
- 
getCheckAndMutateChecksFailedReturns the number of failed checkAndMutate guards
- 
startRegionOperationThis 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
 
- 
startRegionOperationThis 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
 
- 
closeRegionOperationCloses the region operation lock.- Throws:
- IOException
 
- 
closeRegionOperationCloses the region operation lock. This needs to be called in the finally block corresponding to the try block ofstartRegionOperation(Operation)- Throws:
- IOException
 
- 
getRowLockGet 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 against
- readLock- is the lock reader or writer. True indicates that a non-exclusive lock is requested
- Throws:
- IOException
- See Also:
 
- 
appendPerform one or more append operations on a row.- Returns:
- result of the operation
- Throws:
- IOException
 
- 
batchMutatePerform 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 check
- family- column family to check
- qualifier- column qualifier to check
- op- the comparison operator
- comparator- the expected value
- mutation- 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 check
- family- column family to check
- qualifier- column qualifier to check
- op- the comparison operator
- comparator- the expected value
- mutation- data to put if check succeeds
- timeRange- 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 check
- filter- the filter
- mutation- 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 check
- filter- the filter
- mutation- data to put if check succeeds
- timeRange- 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 check
- family- column family to check
- qualifier- column qualifier to check
- op- the comparison operator
- comparator- the expected value
- mutations- 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 check
- family- column family to check
- qualifier- column qualifier to check
- op- the comparison operator
- comparator- the expected value
- mutations- data to put if check succeeds
- timeRange- 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 check
- filter- the filter
- mutations- 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 check
- filter- the filter
- mutations- data to put if check succeeds
- timeRange- time range to check
- Returns:
- true if mutations were applied, false otherwise
- Throws:
- IOException
 
- 
checkAndMutateAtomically 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
 
- 
deleteDeletes the specified cells/row.- Throws:
- IOException
 
- 
getDo a get based on the get parameter.- Parameters:
- get- query parameters
- Returns:
- result of the operation
- Throws:
- IOException
 
- 
getDo a get based on the get parameter.- Parameters:
- get- query parameters
- withCoprocessor- invoke coprocessor or not. We don't want to always invoke cp.
- Returns:
- list of cells resulting from the operation
- Throws:
- IOException
 
- 
getScannerReturn 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- configured- Scan
- Throws:
- IOException- read exceptions
 
- 
getScannerReturn 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- configured- Scan
- additionalScanners- Any additional scanners to be used
- Throws:
- IOException- read exceptions
 
- 
getCellComparatorThe comparator to be used with the region
- 
incrementPerform one or more increment operations on a row.- Returns:
- result of the operation
- Throws:
- IOException
 
- 
mutateRowPerforms 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
 
- 
mutateRowsWithLocksvoid 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.- mutationscan contain operations for multiple rows. Caller has to ensure that all rows are contained in this region.
- rowsToLock- Rows to lock
- nonceGroup- 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- rowsToLockis sorted in order to avoid deadlocks.
- Throws:
- IOException
 
- 
putPuts some data in the table.- Throws:
- IOException
 
- 
getCompactionStateReturns if a given region is in compaction now.
- 
requestCompactionvoid requestCompaction(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException Request compaction on this region.- Throws:
- IOException
 
- 
requestCompactionvoid requestCompaction(byte[] family, String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException Request compaction for the given family- Throws:
- IOException
 
- 
requestFlushRequest flush on this region.- Throws:
- IOException
 
- 
waitForFlushesWait 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.
 
- 
getReadOnlyConfigurationorg.apache.hadoop.conf.Configuration getReadOnlyConfiguration()- Returns:
- a read only configuration of this region; throws UnsupportedOperationExceptionif you try to set a configuration.
 
- 
getMinBlockSizeBytesint getMinBlockSizeBytes()The minimum block size configuration from all relevant column families. This is used when estimating quota consumption.
 
-