@InterfaceAudience.Private public class HRegion extends Object implements HeapSize, PropagatingConfigurationObserver, Region
| Modifier and Type | Class and Description |
|---|---|
static class |
HRegion.FlushResultImpl
Objects from this class are created when flushing to describe all the different states that
that method ends up in.
|
static class |
HRegion.RowLockImpl
Class used to represent a lock on a row.
|
Region.BulkLoadListener, Region.FlushResult, Region.Operation, Region.RowLock| Modifier and Type | Field and Description |
|---|---|
protected org.apache.hadoop.conf.Configuration |
conf |
static long |
DEEP_OVERHEAD |
static int |
DEFAULT_CACHE_FLUSH_INTERVAL
Default interval for the memstore flush
|
static long |
DEFAULT_FLUSH_PER_CHANGES |
static boolean |
DEFAULT_HREGION_MVCC_PRE_ASSIGN |
static int |
DEFAULT_MAX_CELL_SIZE |
static long |
FIXED_OVERHEAD |
static String |
HBASE_MAX_CELL_SIZE_KEY |
static String |
HREGION_MVCC_PRE_ASSIGN
Config key for using mvcc pre-assign feature for put
|
protected long |
lastReplayedCompactionSeqId |
protected long |
lastReplayedOpenRegionSeqId
The sequence id of the last replayed open region event from the primary region.
|
static String |
LOAD_CFS_ON_DEMAND_CONFIG_KEY |
static long |
MAX_FLUSH_PER_CHANGES
The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes
overhead.
|
static String |
MEMSTORE_FLUSH_PER_CHANGES
Conf key to force a flush if there are already enough changes for one region in memstore
|
static String |
MEMSTORE_PERIODIC_FLUSH_INTERVAL
Conf key for the periodic flush interval
|
protected Map<byte[],Store> |
stores |
static int |
SYSTEM_CACHE_FLUSH_INTERVAL
Default interval for System tables memstore flush
|
| Constructor and Description |
|---|
HRegion(HRegionFileSystem fs,
WAL wal,
org.apache.hadoop.conf.Configuration confParam,
HTableDescriptor htd,
RegionServerServices rsServices)
HRegion constructor.
|
HRegion(org.apache.hadoop.fs.Path tableDir,
WAL wal,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.conf.Configuration confParam,
HRegionInfo regionInfo,
HTableDescriptor htd,
RegionServerServices rsServices)
Deprecated.
Use other constructors.
|
| Modifier and Type | Method and Description |
|---|---|
long |
addAndGetGlobalMemstoreSize(long memStoreSize)
Increase the size of mem store in this region and the size of global mem
store
|
static void |
addRegionToMETA(HRegion meta,
HRegion r)
Inserts a new region's meta information into the passed
meta region. |
void |
addRegionToSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription desc,
ForeignExceptionSnare exnSnare)
Complete taking the snapshot on the region.
|
Result |
append(Append append) |
Result |
append(Append mutate,
long nonceGroup,
long nonce)
Perform one or more append operations on a row.
|
boolean |
areWritesEnabled() |
OperationStatus[] |
batchMutate(Mutation[] mutations) |
OperationStatus[] |
batchMutate(Mutation[] mutations,
long nonceGroup,
long nonce)
Perform a batch of mutations.
|
OperationStatus[] |
batchReplay(WALSplitter.MutationReplay[] mutations,
long replaySeqId)
Replay a batch of mutations.
|
boolean |
bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths,
boolean assignSeqId,
Region.BulkLoadListener bulkLoadListener)
Deprecated.
|
boolean |
bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths,
boolean assignSeqId,
Region.BulkLoadListener bulkLoadListener,
List<String> clusterIds)
Attempts to atomically load a group of hfiles.
|
boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Mutation w,
boolean writeToWAL)
Atomically checks if a row/family/qualifier value matches the expected val
If it does, it performs the row mutations.
|
boolean |
checkAndRowMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
RowMutations rm,
boolean writeToWAL)
Atomically checks if a row/family/qualifier value matches the expected val
If it does, it performs the row mutations.
|
void |
checkFamilies(Collection<byte[]> families)
Check the collection of families for validity.
|
protected void |
checkReadOnly() |
protected void |
checkReadsEnabled() |
byte[] |
checkSplit()
Return the splitpoint.
|
void |
checkTimestamps(Map<byte[],List<Cell>> familyMap,
long now)
Check the collection of families for valid timestamps
|
Map<byte[],List<StoreFile>> |
close()
Close down this HRegion.
|
Map<byte[],List<StoreFile>> |
close(boolean abort)
Close down this HRegion.
|
static void |
closeHRegion(HRegion r)
This will do the necessary cleanup a call to
createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)
requires. |
void |
closeRegionOperation()
Closes the region operation lock.
|
void |
closeRegionOperation(Region.Operation operation)
Closes the region operation lock.
|
void |
compact(boolean majorCompaction)
Synchronously compact all stores in the region.
|
boolean |
compact(CompactionContext compaction,
Store store,
ThroughputController throughputController) |
boolean |
compact(CompactionContext compaction,
Store store,
ThroughputController throughputController,
User user) |
void |
compactStores()
This is a helper function that compact all the stores synchronously
It is used by utilities and testing
|
static HDFSBlocksDistribution |
computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor tableDescriptor,
HRegionInfo regionInfo)
This is a helper function to compute HDFS block distribution on demand
|
static HDFSBlocksDistribution |
computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor tableDescriptor,
HRegionInfo regionInfo,
org.apache.hadoop.fs.Path tablePath)
This is a helper function to compute HDFS block distribution on demand
|
static HRegion |
createHRegion(HRegionInfo info,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.conf.Configuration conf,
HTableDescriptor hTableDescriptor)
Convenience method creating new HRegions.
|
static HRegion |
createHRegion(HRegionInfo info,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.conf.Configuration conf,
HTableDescriptor hTableDescriptor,
WAL wal) |
static HRegion |
createHRegion(HRegionInfo info,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.conf.Configuration conf,
HTableDescriptor hTableDescriptor,
WAL wal,
boolean initialize)
Convenience method creating new HRegions.
|
static HRegion |
createHRegion(HRegionInfo info,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.conf.Configuration conf,
HTableDescriptor hTableDescriptor,
WAL wal,
boolean initialize,
boolean ignoreWAL)
Convenience method creating new HRegions.
|
static HRegion |
createHRegion(HRegionInfo info,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.fs.Path tableDir,
org.apache.hadoop.conf.Configuration conf,
HTableDescriptor hTableDescriptor,
WAL wal,
boolean initialize,
boolean ignoreWAL)
Convenience method creating new HRegions.
|
void |
decrementCompactionsQueuedCount() |
void |
delete(Delete delete)
Deletes the specified cells/row.
|
void |
deregisterChildren(ConfigurationManager manager)
Needs to be called to deregister the children from the manager.
|
protected void |
doRegionCompactionPrep() |
boolean |
equals(Object o) |
com.google.protobuf.Message |
execService(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall call)
Executes a single protocol buffer coprocessor endpoint
Service method using
the registered protocol handlers. |
Region.FlushResult |
flush(boolean force)
Flush the cache.
|
Region.FlushResult |
flushcache(boolean forceFlushAllStores,
boolean writeFlushRequestWalMarker)
Flush the cache.
|
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.
|
List<Cell> |
get(Get get,
boolean withCoprocessor,
long nonceGroup,
long nonce)
Do a get for duplicate non-idempotent operation.
|
long |
getBlockedRequestsCount() |
long |
getCheckAndMutateChecksFailed() |
long |
getCheckAndMutateChecksPassed() |
Result |
getClosestRowBefore(byte[] row,
byte[] family)
Return all the data for the row that matches row exactly,
or the one that immediately preceeds it, at or immediately before
ts.
|
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionState() |
int |
getCompactPriority() |
KeyValue.KVComparator |
getComparator() |
RegionCoprocessorHost |
getCoprocessorHost() |
long |
getDataInMemoryWithoutWAL() |
long |
getEarliestFlushTimeForAllStores() |
protected Durability |
getEffectiveDurability(Durability d)
Returns effective durability from the passed durability and
the table descriptor.
|
org.apache.hadoop.fs.FileSystem |
getFilesystem() |
HDFSBlocksDistribution |
getHDFSBlocksDistribution() |
long |
getLargestHStoreSize() |
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats |
getLoadStatistics() |
ConcurrentHashMap<HashedBytes,org.apache.hadoop.hbase.regionserver.HRegion.RowLockContext> |
getLockedRows() |
long |
getMaxFlushedSeqId() |
Map<byte[],Long> |
getMaxStoreSeqId() |
long |
getMemstoreSize() |
MetricsRegion |
getMetrics() |
MultiVersionConcurrencyControl |
getMVCC() |
protected long |
getNextSequenceId(WAL wal)
Method to safely get the next sequence number.
|
long |
getNumMutationsWithoutWAL() |
long |
getOldestHfileTs(boolean majorCompactionOnly)
This can be used to determine the last time all files of this region were major compacted.
|
long |
getOldestSeqIdOfStore(byte[] familyName) |
long |
getOpenSeqNum() |
long |
getReadpoint(IsolationLevel isolationLevel) |
long |
getReadRequestsCount() |
HRegionFileSystem |
getRegionFileSystem() |
HRegionInfo |
getRegionInfo() |
Region.RowLock |
getRowLock(byte[] row)
Get an exclusive ( write lock ) lock on a given row.
|
Region.RowLock |
getRowLock(byte[] row,
boolean readLock)
Get a row lock for the specified row.
|
Region.RowLock |
getRowLock(byte[] row,
boolean readLock,
boolean waitForLock)
Get a row lock for the specified row.
|
protected Region.RowLock |
getRowLockInternal(byte[] row) |
protected Region.RowLock |
getRowLockInternal(byte[] row,
boolean readLock,
boolean waitForLock,
Region.RowLock prevRowLock) |
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. |
long |
getSequenceId()
Do not change this sequence id.
|
long |
getSmallestReadPoint() |
RegionSplitPolicy |
getSplitPolicy() |
Store |
getStore(byte[] column)
Return the Store for the given family
|
List<String> |
getStoreFileList(byte[][] columns) |
protected ThreadPoolExecutor |
getStoreFileOpenAndCloseThreadPool(String threadNamePrefix) |
protected ThreadPoolExecutor |
getStoreOpenAndCloseThreadPool(String threadNamePrefix) |
List<Store> |
getStores()
Return the list of Stores managed by this region
|
HTableDescriptor |
getTableDesc() |
WAL |
getWAL() |
org.apache.hadoop.fs.Path |
getWALRegionDir() |
long |
getWriteRequestsCount() |
int |
hashCode() |
boolean |
hasReferences() |
long |
heapSize() |
Result |
increment(Increment increment) |
Result |
increment(Increment mutation,
long nonceGroup,
long nonce)
Perform one or more increment operations on a row.
|
void |
incrementCompactionsQueuedCount() |
void |
incrementFlushesQueuedCount() |
long |
initialize()
Deprecated.
use HRegion.createHRegion() or HRegion.openHRegion()
|
protected HStore |
instantiateHStore(HColumnDescriptor family) |
protected RegionScanner |
instantiateRegionScanner(Scan scan,
List<KeyValueScanner> additionalScanners) |
protected RegionScanner |
instantiateRegionScanner(Scan scan,
List<KeyValueScanner> additionalScanners,
long nonceGroup,
long nonce) |
protected Region.FlushResult |
internalFlushcache(WAL wal,
long myseqid,
Collection<Store> storesToFlush,
MonitoredTask status,
boolean writeFlushWalMarker)
Flush the memstore.
|
protected Region.FlushResult |
internalFlushCacheAndCommit(WAL wal,
MonitoredTask status,
org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult prepareResult,
Collection<Store> storesToFlush) |
protected org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult |
internalPrepareFlushCache(WAL wal,
long myseqid,
Collection<Store> storesToFlush,
MonitoredTask status,
boolean writeFlushWalMarker) |
boolean |
isAvailable() |
boolean |
isClosed() |
boolean |
isClosing() |
boolean |
isLoadingCfsOnDemandDefault() |
boolean |
isMergeable() |
boolean |
isReadOnly() |
boolean |
isRecovering() |
boolean |
isSplittable() |
static void |
main(String[] args)
Facility for dumping and compacting catalog tables.
|
static HRegion |
merge(HRegion a,
HRegion b)
Merge two regions whether they are adjacent or not.
|
static HRegion |
mergeAdjacent(HRegion srcA,
HRegion srcB)
Merge two HRegions.
|
void |
mutateRow(RowMutations rm)
Performs multiple mutations atomically on a single row.
|
void |
mutateRowsWithLocks(Collection<Mutation> mutations,
Collection<byte[]> rowsToLock)
Perform atomic mutations within the region w/o nonces.
|
void |
mutateRowsWithLocks(Collection<Mutation> mutations,
Collection<byte[]> rowsToLock,
long nonceGroup,
long nonce)
Perform atomic mutations within the region.
|
void |
onConfigurationChange(org.apache.hadoop.conf.Configuration conf)
This method would be called by the
ConfigurationManager
object when the Configuration object is reloaded from disk. |
protected HRegion |
openHRegion(CancelableProgressable reporter)
Open HRegion.
|
static HRegion |
openHRegion(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path rootDir,
HRegionInfo info,
HTableDescriptor htd,
WAL wal)
Open a Region.
|
static HRegion |
openHRegion(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path rootDir,
HRegionInfo info,
HTableDescriptor htd,
WAL wal,
RegionServerServices rsServices,
CancelableProgressable reporter)
Open a Region.
|
static HRegion |
openHRegion(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path rootDir,
org.apache.hadoop.fs.Path tableDir,
HRegionInfo info,
HTableDescriptor htd,
WAL wal,
RegionServerServices rsServices,
CancelableProgressable reporter)
Open a Region.
|
static HRegion |
openHRegion(HRegion other,
CancelableProgressable reporter)
Useful when reopening a closed region (normally for unit tests)
|
static HRegion |
openHRegion(HRegionInfo info,
HTableDescriptor htd,
WAL wal,
org.apache.hadoop.conf.Configuration conf)
Open a Region.
|
static HRegion |
openHRegion(HRegionInfo info,
HTableDescriptor htd,
WAL wal,
org.apache.hadoop.conf.Configuration conf,
RegionServerServices rsServices,
CancelableProgressable reporter)
Open a Region.
|
static HRegion |
openHRegion(org.apache.hadoop.fs.Path rootDir,
HRegionInfo info,
HTableDescriptor htd,
WAL wal,
org.apache.hadoop.conf.Configuration conf)
Open a Region.
|
static HRegion |
openHRegion(org.apache.hadoop.fs.Path rootDir,
HRegionInfo info,
HTableDescriptor htd,
WAL wal,
org.apache.hadoop.conf.Configuration conf,
RegionServerServices rsServices,
CancelableProgressable reporter)
Open a Region.
|
static Region |
openHRegion(Region other,
CancelableProgressable reporter) |
static HRegion |
openReadOnlyFileSystemHRegion(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path tableDir,
HRegionInfo info,
HTableDescriptor htd)
Open a Region on a read-only file-system (like hdfs snapshots)
|
void |
prepareDelete(Delete delete)
Prepare a delete for a row mutation processor
|
void |
prepareDeleteTimestamps(Mutation mutation,
Map<byte[],List<Cell>> familyMap,
byte[] byteNow)
Set up correct timestamps in the KVs in Delete object.
|
protected void |
prepareToSplit()
Give the region a chance to prepare before it is split.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor)
Performs atomic multiple reads and writes on a given row.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor,
long nonceGroup,
long nonce)
Performs atomic multiple reads and writes on a given row.
|
void |
processRowsWithLocks(RowProcessor<?,?> processor,
long timeout,
long nonceGroup,
long nonce)
Performs atomic multiple reads and writes on a given row.
|
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 |
registerChildren(ConfigurationManager manager)
Needs to be called to register the children to the manager.
|
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to
be available for handling
Region.execService(com.google.protobuf.RpcController,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)} calls. |
void |
releaseRowLocks(List<Region.RowLock> rowLocks)
If the given list of row locks is not null, releases all locks.
|
protected long |
replayRecoveredEditsIfAny(Map<byte[],Long> maxSeqIdInStores,
CancelableProgressable reporter,
MonitoredTask status)
Read the edits put under this region by wal splitting process.
|
void |
reportCompactionRequestEnd(boolean isMajor,
int numFiles,
long filesSizeCompacted) |
void |
reportCompactionRequestFailure() |
void |
reportCompactionRequestStart(boolean isMajor) |
protected boolean |
restoreEdit(Store s,
Cell cell)
Used by tests
|
static boolean |
rowIsInRange(HRegionInfo info,
byte[] row)
Determines if the specified row is within the row range specified by the
specified HRegionInfo
|
static boolean |
rowIsInRange(HRegionInfo info,
byte[] row,
int offset,
short length) |
void |
setClosing(boolean closing)
Exposed for some very specific unit tests.
|
void |
setCoprocessorHost(RegionCoprocessorHost coprocessorHost) |
void |
setReadsEnabled(boolean readsEnabled) |
void |
setRecovering(boolean newState)
Reset recovering state of current 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.
|
String |
toString() |
void |
triggerMajorCompaction()
Trigger major compaction on all stores in the region.
|
void |
updateCellTimestamps(Iterable<List<Cell>> cellItr,
byte[] now)
Replace any cell timestamps set to HConstants#LATEST_TIMESTAMP with the
provided current timestamp.
|
void |
updateReadRequestsCount(long i)
Update the read request count for this region
|
void |
updateWriteRequestsCount(long i)
Update the write request count for this region
|
void |
waitForFlushes()
Wait for all current flushes of the region to complete
|
void |
waitForFlushesAndCompactions()
Wait for all current flushes and compactions of the region to complete
|
static void |
warmupHRegion(HRegionInfo info,
HTableDescriptor htd,
WAL wal,
org.apache.hadoop.conf.Configuration conf,
RegionServerServices rsServices,
CancelableProgressable reporter) |
public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY
public static final String HREGION_MVCC_PRE_ASSIGN
public static final boolean DEFAULT_HREGION_MVCC_PRE_ASSIGN
public static final String HBASE_MAX_CELL_SIZE_KEY
public static final int DEFAULT_MAX_CELL_SIZE
protected volatile long lastReplayedOpenRegionSeqId
protected volatile long lastReplayedCompactionSeqId
protected final org.apache.hadoop.conf.Configuration conf
public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL
public static final int DEFAULT_CACHE_FLUSH_INTERVAL
public static final int SYSTEM_CACHE_FLUSH_INTERVAL
public static final String MEMSTORE_FLUSH_PER_CHANGES
public static final long DEFAULT_FLUSH_PER_CHANGES
public static final long MAX_FLUSH_PER_CHANGES
public static final long FIXED_OVERHEAD
public static final long DEEP_OVERHEAD
@Deprecated public HRegion(org.apache.hadoop.fs.Path tableDir, WAL wal, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration confParam, HRegionInfo regionInfo, HTableDescriptor htd, RegionServerServices rsServices)
createHRegion(org.apache.hadoop.hbase.HRegionInfo, org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, org.apache.hadoop.hbase.HTableDescriptor) or openHRegion(org.apache.hadoop.hbase.HRegionInfo, org.apache.hadoop.hbase.HTableDescriptor, org.apache.hadoop.hbase.wal.WAL, org.apache.hadoop.conf.Configuration) method.tableDir - qualified path of directory where region should be located,
usually the table directory.wal - The WAL is the outbound log for any updates to the HRegion
The wal file is a logfile from the previous execution that's
custom-computed for this HRegion. The HRegionServer computes and sorts the
appropriate wal info for this HRegion. If there is a previous wal file
(implying that the HRegion has been written-to before), then read it from
the supplied path.fs - is the filesystem.confParam - is global configuration settings.regionInfo - - HRegionInfo that describes the region
is new), then read them from the supplied path.htd - the table descriptorrsServices - reference to RegionServerServices or nullpublic HRegion(HRegionFileSystem fs, WAL wal, org.apache.hadoop.conf.Configuration confParam, HTableDescriptor htd, RegionServerServices rsServices)
createHRegion(org.apache.hadoop.hbase.HRegionInfo, org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, org.apache.hadoop.hbase.HTableDescriptor) or openHRegion(org.apache.hadoop.hbase.HRegionInfo, org.apache.hadoop.hbase.HTableDescriptor, org.apache.hadoop.hbase.wal.WAL, org.apache.hadoop.conf.Configuration) method.fs - is the filesystem.wal - The WAL is the outbound log for any updates to the HRegion
The wal file is a logfile from the previous execution that's
custom-computed for this HRegion. The HRegionServer computes and sorts the
appropriate wal info for this HRegion. If there is a previous wal file
(implying that the HRegion has been written-to before), then read it from
the supplied path.confParam - is global configuration settings.htd - the table descriptorrsServices - reference to RegionServerServices or nullpublic long getSmallestReadPoint()
@Deprecated public long initialize() throws IOException
IOException - epublic boolean hasReferences()
public HDFSBlocksDistribution getHDFSBlocksDistribution()
getHDFSBlocksDistribution in interface Regionpublic static HDFSBlocksDistribution computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf, HTableDescriptor tableDescriptor, HRegionInfo regionInfo) throws IOException
conf - configurationtableDescriptor - HTableDescriptor of the tableregionInfo - encoded name of the regionIOExceptionpublic static HDFSBlocksDistribution computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf, HTableDescriptor tableDescriptor, HRegionInfo regionInfo, org.apache.hadoop.fs.Path tablePath) throws IOException
conf - configurationtableDescriptor - HTableDescriptor of the tableregionInfo - encoded name of the regiontablePath - the table directoryIOExceptionpublic long addAndGetGlobalMemstoreSize(long memStoreSize)
public HRegionInfo getRegionInfo()
getRegionInfo in interface Regionpublic long getReadRequestsCount()
getReadRequestsCount in interface Regionpublic void updateReadRequestsCount(long i)
RegionupdateReadRequestsCount in interface Regioni - incrementpublic long getWriteRequestsCount()
getWriteRequestsCount in interface Regionpublic void updateWriteRequestsCount(long i)
RegionupdateWriteRequestsCount in interface Regioni - incrementpublic long getMemstoreSize()
getMemstoreSize in interface Regionpublic long getNumMutationsWithoutWAL()
getNumMutationsWithoutWAL in interface Regionpublic long getDataInMemoryWithoutWAL()
getDataInMemoryWithoutWAL in interface Regionpublic long getBlockedRequestsCount()
getBlockedRequestsCount in interface Regionpublic long getCheckAndMutateChecksPassed()
getCheckAndMutateChecksPassed in interface Regionpublic long getCheckAndMutateChecksFailed()
getCheckAndMutateChecksFailed in interface Regionpublic MetricsRegion getMetrics()
getMetrics in interface Regionpublic boolean isClosed()
public boolean isClosing()
public boolean isReadOnly()
isReadOnly in interface Regionpublic void setRecovering(boolean newState)
public boolean isRecovering()
isRecovering in interface Regionpublic boolean isAvailable()
isAvailable in interface Regionpublic boolean isSplittable()
public boolean isMergeable()
public boolean areWritesEnabled()
public MultiVersionConcurrencyControl getMVCC()
public long getMaxFlushedSeqId()
getMaxFlushedSeqId in interface Regionpublic long getReadpoint(IsolationLevel isolationLevel)
getReadpoint in interface Regionpublic boolean isLoadingCfsOnDemandDefault()
isLoadingCfsOnDemandDefault in interface Regionpublic Map<byte[],List<StoreFile>> close() throws IOException
This method could take some time to execute, so don't call it from a time-sensitive thread.
IOException - eDroppedSnapshotException - Thrown when replay of wal is required
because a Snapshot was not properly persisted. The region is put in closing mode, and the
caller MUST abort after this.public Map<byte[],List<StoreFile>> close(boolean abort) throws IOException
abort - true if server is aborting (only during testing)IOException - eDroppedSnapshotException - Thrown when replay of wal is required
because a Snapshot was not properly persisted. The region is put in closing mode, and the
caller MUST abort after this.public void setClosing(boolean closing)
public void waitForFlushesAndCompactions()
RegionwaitForFlushesAndCompactions in interface Regionpublic void waitForFlushes()
RegionwaitForFlushes in interface Regionprotected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(String threadNamePrefix)
protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(String threadNamePrefix)
public HTableDescriptor getTableDesc()
getTableDesc in interface Regionpublic WAL getWAL()
public org.apache.hadoop.fs.FileSystem getFilesystem()
FileSystem being used by this regionpublic HRegionFileSystem getRegionFileSystem()
HRegionFileSystem used by this regionpublic org.apache.hadoop.fs.Path getWALRegionDir()
throws IOException
IOException - if there is an error getting WALRootDirpublic long getEarliestFlushTimeForAllStores()
getEarliestFlushTimeForAllStores in interface Regionpublic long getOldestHfileTs(boolean majorCompactionOnly)
throws IOException
RegiongetOldestHfileTs in interface RegionmajorCompactionOnly - Only consider HFile that are the result of major compactionIOExceptionpublic long getLargestHStoreSize()
public KeyValue.KVComparator getComparator()
protected void doRegionCompactionPrep()
throws IOException
IOExceptionpublic void triggerMajorCompaction()
throws IOException
Region
Compaction will be performed asynchronously to this call by the RegionServer's
CompactSplitThread. See also Store.triggerMajorCompaction()
triggerMajorCompaction in interface RegionIOExceptionpublic void compact(boolean majorCompaction)
throws IOException
RegionThis operation could block for a long time, so don't call it from a time-sensitive thread.
Note that no locks are taken to prevent possible conflicts between compaction and splitting activities. The regionserver does not normally compact and split in parallel. However by calling this method you may introduce unexpected and unhandled concurrency. Don't do this unless you know what you are doing.
compact in interface RegionmajorCompaction - True to force a major compaction regardless of thresholdsIOExceptionpublic void compactStores()
throws IOException
IOException - epublic boolean compact(CompactionContext compaction, Store store, ThroughputController throughputController) throws IOException
IOExceptionpublic boolean compact(CompactionContext compaction, Store store, ThroughputController throughputController, User user) throws IOException
IOExceptionpublic Region.FlushResult flush(boolean force) throws IOException
RegionWhen this method is called the cache will be flushed unless:
This method may block for some time, so it should not be called from a time-sensitive thread.
flush in interface Regionforce - whether we want to force a flush of all storesIOException - general io exceptions
because a snapshot was not properly persisted.public Region.FlushResult flushcache(boolean forceFlushAllStores, boolean writeFlushRequestWalMarker) throws IOException
This method may block for some time, so it should not be called from a time-sensitive thread.
forceFlushAllStores - whether we want to flush all storeswriteFlushRequestWalMarker - whether to write the flush request marker to WALIOException - general io exceptionsDroppedSnapshotException - Thrown when replay of wal is required
because a Snapshot was not properly persisted. The region is put in closing mode, and the
caller MUST abort after this.protected Region.FlushResult internalFlushcache(WAL wal, long myseqid, Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker) throws IOException
This method may block for some time. Every time you call it, we up the regions sequence id even if we don't flush; i.e. the returned region id will be at least one larger than the last edit applied to this region. The returned id does not refer to an actual edit. The returned id can be used for say installing a bulk loaded file just ahead of the last hfile that was the result of this flush, etc.
wal - Null if we're NOT to go via wal.myseqid - The seqid to use if wal is null writing out flush
file.storesToFlush - The list of stores to flush.IOException - general io exceptionsDroppedSnapshotException - Thrown when replay of wal is required because a Snapshot was not
properly persisted.protected org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid, Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker) throws IOException
IOExceptionprotected Region.FlushResult internalFlushCacheAndCommit(WAL wal, MonitoredTask status, org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult prepareResult, Collection<Store> storesToFlush) throws IOException
IOExceptionprotected long getNextSequenceId(WAL wal) throws IOException
IOExceptionpublic Result getClosestRowBefore(byte[] row, byte[] family) throws IOException
RegiongetClosestRowBefore in interface RegionIOExceptionpublic RegionScanner getScanner(Scan scan) throws IOException
RegionScan.
This Iterator must be closed by the caller.
getScanner in interface Regionscan - configured ScanIOException - read exceptionspublic RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException
RegionScan. 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.
getScanner in interface Regionscan - configured ScanadditionalScanners - Any additional scanners to be usedIOException - read exceptionsprotected RegionScanner instantiateRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException
IOExceptionprotected RegionScanner instantiateRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException
IOExceptionpublic void prepareDelete(Delete delete) throws IOException
RegionprepareDelete in interface Regiondelete - The passed delete is modified by this method. WARNING!IOExceptionpublic void delete(Delete delete) throws IOException
Regiondelete in interface RegionIOExceptionpublic void prepareDeleteTimestamps(Mutation mutation, Map<byte[],List<Cell>> familyMap, byte[] byteNow) throws IOException
RegionCaller should have the row and region locks.
prepareDeleteTimestamps in interface RegionIOExceptionpublic void put(Put put) throws IOException
Regionput in interface RegionIOExceptionpublic OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce) throws IOException
RegionNote this supports only Put and Delete mutations and will ignore other types passed.
batchMutate in interface Regionmutations - the list of mutationsIOExceptionpublic OperationStatus[] batchMutate(Mutation[] mutations) throws IOException
IOExceptionpublic OperationStatus[] batchReplay(WALSplitter.MutationReplay[] mutations, long replaySeqId) throws IOException
RegionbatchReplay in interface Regionmutations - mutations to replay.IOExceptionprotected Durability getEffectiveDurability(Durability d)
public boolean checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Mutation w,
boolean writeToWAL)
throws IOException
RegioncheckAndMutate in interface Regionrow - to checkfamily - column family to checkqualifier - column qualifier to checkcompareOp - the comparison operatorIOExceptionpublic boolean checkAndRowMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
RowMutations rm,
boolean writeToWAL)
throws IOException
RegioncheckAndRowMutate in interface Regionrow - to checkfamily - column family to checkqualifier - column qualifier to checkcompareOp - the comparison operatorIOExceptionpublic void addRegionToSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription desc,
ForeignExceptionSnare exnSnare)
throws IOException
ForeignExceptionSnare
arg. (In the future other cancellable HRegion methods could eventually add a
ForeignExceptionSnare, or we could do something fancier).desc - snapshot description objectexnSnare - ForeignExceptionSnare that captures external exceptions in case we need to
bail out. This is allowed to be null and will just be ignored in that case.IOException - if there is an external or internal error causing the snapshot to failpublic void updateCellTimestamps(Iterable<List<Cell>> cellItr, byte[] now) throws IOException
RegionupdateCellTimestamps in interface RegionIOExceptionprotected void checkReadOnly()
throws IOException
IOException - Throws exception if region is in read-only mode.protected void checkReadsEnabled()
throws IOException
IOExceptionpublic void setReadsEnabled(boolean readsEnabled)
public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException
RegioncheckFamilies in interface RegionNoSuchColumnFamilyExceptionpublic void checkTimestamps(Map<byte[],List<Cell>> familyMap, long now) throws FailedSanityCheckException
RegioncheckTimestamps in interface Regionnow - current timestampFailedSanityCheckExceptionprotected long replayRecoveredEditsIfAny(Map<byte[],Long> maxSeqIdInStores, CancelableProgressable reporter, MonitoredTask status) throws IOException
We can ignore any wal message that has a sequence ID that's equal to or lower than minSeqId. (Because we know such messages are already reflected in the HFiles.)
While this is running we are putting pressure on memory yet we are outside of our usual accounting because we are not yet an onlined region (this stuff is being run as part of Region initialization). This means that if we're up against global memory limits, we'll not be flagged to flush because we are not online. We can't be flushed by usual mechanisms anyways; we're not yet online so our relative sequenceids are not yet aligned with WAL sequenceids -- not till we come up online, post processing of split edits.
But to help relieve memory pressure, at least manage our own heap size flushing if are in excess of per-region limits. Flushing, though, we have to be careful and avoid using the regionserver/wal sequenceid. Its running on a different line to whats going on in here in this region context so if we crashed replaying these edits, but in the midst had a flush that used the regionserver wal with a sequenceid in excess of whats going on in here in this region and with its split editlogs, then we could miss edits the next time we go to recover. So, we have to flush inline, using seqids that make sense in a this single region context only -- until we online.
maxSeqIdInStores - Any edit found in split editlogs needs to be in excess of
the maxSeqId for the store to be applied, else its skipped.minSeqId if nothing added from editlogs.IOExceptionpublic boolean refreshStoreFiles()
throws IOException
RegionrefreshStoreFiles in interface RegionIOExceptionprotected boolean restoreEdit(Store s, Cell cell)
s - Store to add edit too.cell - Cell to add.protected HStore instantiateHStore(HColumnDescriptor family) throws IOException
IOExceptionpublic Store getStore(byte[] column)
RegionUse with caution. Exposed for use of fixup utilities.
public List<Store> getStores()
RegionUse with caution. Exposed for use of fixup utilities.
public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException
getStoreFileList in interface RegionIllegalArgumentExceptionpublic Region.RowLock getRowLock(byte[] row) throws IOException
row - Which row to lock.IOException - if any error occurredpublic Region.RowLock getRowLock(byte[] row, boolean readLock) throws IOException
RegiongetRowLock in interface Regionrow - The row actions will be performed againstreadLock - is the lock reader or writer. True indicates that a non-exclusive
lock is requestedIOExceptionRegion.startRegionOperation(),
Region.startRegionOperation(Operation)public Region.RowLock getRowLock(byte[] row, boolean readLock, boolean waitForLock) throws IOException
row - The row actions will be performed againstreadLock - is the lock reader or writer. True indicates that a non-exlcusive
lock is requestedwaitForLock - whether should wait for this lockwaitForLock set to false and tryLock failedIOException - if any error occurredprotected Region.RowLock getRowLockInternal(byte[] row) throws IOException
IOExceptionprotected Region.RowLock getRowLockInternal(byte[] row, boolean readLock, boolean waitForLock, Region.RowLock prevRowLock) throws IOException
IOExceptionpublic void releaseRowLocks(List<Region.RowLock> rowLocks)
RegionreleaseRowLocks in interface Regionpublic ConcurrentHashMap<HashedBytes,org.apache.hadoop.hbase.regionserver.HRegion.RowLockContext> getLockedRows()
public boolean bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths, boolean assignSeqId, Region.BulkLoadListener bulkLoadListener, List<String> clusterIds) throws IOException
RegionbulkLoadHFiles in interface RegionfamilyPaths - List of Pair<byte[] column family, String hfilePath>bulkLoadListener - Internal hooks enabling massaging/preparation of a
file about to be bulk loadedIOException - if failed unrecoverably.@Deprecated public boolean bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths, boolean assignSeqId, Region.BulkLoadListener bulkLoadListener) throws IOException
RegionbulkLoadHFiles in interface RegionfamilyPaths - List of Pair<byte[] column family, String hfilePath>bulkLoadListener - Internal hooks enabling massaging/preparation of a
file about to be bulk loadedIOException - if failed unrecoverably.public static HRegion createHRegion(HRegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, HTableDescriptor hTableDescriptor) throws IOException
WAL for the created region. It
needs to be closed explicitly. Use getWAL() to get
access. When done with a region created using this method, you will
need to explicitly close the WAL it created too; it will not be
done for you. Not closing the wal will leave at least a daemon thread
running. Call closeHRegion(HRegion) and it will do
necessary cleanup for you.info - Info for region to create.rootDir - Root directory for HBase instanceIOExceptionpublic static void closeHRegion(HRegion r) throws IOException
createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)
requires. This method will close the region and then close its
associated WAL file. You can still use it if you call the other createHRegion,
the one that takes an WAL instance but don't be surprised by the
call to the WAL.close() on the WAL the
HRegion was carrying.IOExceptionpublic static HRegion createHRegion(HRegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, HTableDescriptor hTableDescriptor, WAL wal, boolean initialize) throws IOException
WAL for the created region needs to be closed explicitly.
Use getWAL() to get access.info - Info for region to create.rootDir - Root directory for HBase instancewal - shared WALinitialize - - true to initialize the regionIOExceptionpublic static HRegion createHRegion(HRegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, HTableDescriptor hTableDescriptor, WAL wal, boolean initialize, boolean ignoreWAL) throws IOException
WAL for the created region needs to be closed
explicitly, if it is not null.
Use getWAL() to get access.info - Info for region to create.rootDir - Root directory for HBase instancewal - shared WALinitialize - - true to initialize the regionignoreWAL - - true to skip generate new wal if it is null, mostly for createTableIOExceptionpublic static HRegion createHRegion(HRegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.fs.Path tableDir, org.apache.hadoop.conf.Configuration conf, HTableDescriptor hTableDescriptor, WAL wal, boolean initialize, boolean ignoreWAL) throws IOException
WAL for the created region needs to be closed
explicitly, if it is not null.
Use getWAL() to get access.info - Info for region to create.rootDir - Root directory for HBase instancetableDir - table directorywal - shared WALinitialize - - true to initialize the regionignoreWAL - - true to skip generate new wal if it is null, mostly for createTableIOExceptionpublic static HRegion createHRegion(HRegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, HTableDescriptor hTableDescriptor, WAL wal) throws IOException
IOExceptionpublic static HRegion openHRegion(HRegionInfo info, HTableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf) throws IOException
info - Info for region to be opened.wal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.IOExceptionpublic static HRegion openHRegion(HRegionInfo info, HTableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
info - Info for region to be openedhtd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.conf - The Configuration object to use.rsServices - An interface we can request flushes against.reporter - An interface we can report progress against.IOExceptionpublic static HRegion openHRegion(org.apache.hadoop.fs.Path rootDir, HRegionInfo info, HTableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf) throws IOException
rootDir - Root directory for HBase instanceinfo - Info for region to be opened.htd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.conf - The Configuration object to use.IOExceptionpublic static HRegion openHRegion(org.apache.hadoop.fs.Path rootDir, HRegionInfo info, HTableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
rootDir - Root directory for HBase instanceinfo - Info for region to be opened.htd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.conf - The Configuration object to use.rsServices - An interface we can request flushes against.reporter - An interface we can report progress against.IOExceptionpublic static HRegion openHRegion(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path rootDir, HRegionInfo info, HTableDescriptor htd, WAL wal) throws IOException
conf - The Configuration object to use.fs - Filesystem to userootDir - Root directory for HBase instanceinfo - Info for region to be opened.htd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.IOExceptionpublic static HRegion openHRegion(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path rootDir, HRegionInfo info, HTableDescriptor htd, WAL wal, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
conf - The Configuration object to use.fs - Filesystem to userootDir - Root directory for HBase instanceinfo - Info for region to be opened.htd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.rsServices - An interface we can request flushes against.reporter - An interface we can report progress against.IOExceptionpublic static HRegion openHRegion(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.fs.Path tableDir, HRegionInfo info, HTableDescriptor htd, WAL wal, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
conf - The Configuration object to use.fs - Filesystem to userootDir - Root directory for HBase instanceinfo - Info for region to be opened.htd - the table descriptorwal - WAL for region to use. This method will call
WAL#setSequenceNumber(long) passing the result of the call to
HRegion#getMinSequenceId() to ensure the wal id is properly kept
up. HRegionStore does this every time it opens a new region.rsServices - An interface we can request flushes against.reporter - An interface we can report progress against.IOExceptionpublic static HRegion openHRegion(HRegion other, CancelableProgressable reporter) throws IOException
other - original objectreporter - An interface we can report progress against.IOExceptionpublic static Region openHRegion(Region other, CancelableProgressable reporter) throws IOException
IOExceptionprotected HRegion openHRegion(CancelableProgressable reporter) throws IOException
thisIOExceptionpublic static HRegion openReadOnlyFileSystemHRegion(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, HRegionInfo info, HTableDescriptor htd) throws IOException
conf - The Configuration object to use.fs - Filesystem to useinfo - Info for region to be opened.htd - the table descriptorIOException - epublic static void warmupHRegion(HRegionInfo info, HTableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
IOExceptionpublic static void addRegionToMETA(HRegion meta, HRegion r) throws IOException
meta region. Used by the HMaster bootstrap code adding
new table to hbase:meta table.meta - hbase:meta HRegion to be updatedr - HRegion to add to metaIOExceptionpublic static boolean rowIsInRange(HRegionInfo info, byte[] row)
info - HRegionInfo that specifies the row rangerow - row to be checkedpublic static boolean rowIsInRange(HRegionInfo info, byte[] row, int offset, short length)
public static HRegion mergeAdjacent(HRegion srcA, HRegion srcB) throws IOException
IOExceptionpublic static HRegion merge(HRegion a, HRegion b) throws IOException
a - region ab - region bIOExceptionpublic Result get(Get get) throws IOException
Regionget in interface Regionget - query parametersIOExceptionpublic List<Cell> get(Get get, boolean withCoprocessor) throws IOException
Regionget in interface Regionget - query parameterswithCoprocessor - invoke coprocessor or not. We don't want to
always invoke cp.IOExceptionpublic List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) throws IOException
Regionget in interface Regionget - query parameters.nonceGroup - Nonce group.nonce - Nonce.IOExceptionpublic void mutateRow(RowMutations rm) throws IOException
RegionmutateRow in interface Regionrm - object that specifies the set of mutations to perform atomicallyIOExceptionpublic void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock) throws IOException
mutateRowsWithLocks(Collection, Collection, long, long)IOExceptionpublic void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException
mutateRowsWithLocks in interface Regionmutations - 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.IOExceptionpublic org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats getLoadStatistics()
public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException
RegionprocessRowsWithLocks in interface Regionprocessor - The object defines the reads and writes to a row.IOExceptionpublic void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce) throws IOException
RegionprocessRowsWithLocks in interface Regionprocessor - 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")IOExceptionpublic void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, long nonceGroup, long nonce) throws IOException
RegionprocessRowsWithLocks in interface Regionprocessor - 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")IOExceptionpublic Result append(Append append) throws IOException
IOExceptionpublic Result append(Append mutate, long nonceGroup, long nonce) throws IOException
Regionappend in interface RegionIOExceptionpublic Result increment(Increment increment) throws IOException
IOExceptionpublic Result increment(Increment mutation, long nonceGroup, long nonce) throws IOException
Regionincrement in interface RegionIOExceptionpublic long heapSize()
public boolean registerService(com.google.protobuf.Service instance)
RegionService subclass as a coprocessor endpoint to
be available for handling
Region.execService(com.google.protobuf.RpcController,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)} calls.
Only a single instance may be registered per region for a given Service subclass (the
instances are keyed on Descriptors.ServiceDescriptor.getFullName().
After the first registration, subsequent calls with the same service name will fail with
a return value of false.
registerService in interface Regioninstance - the Service subclass instance to expose as a coprocessor endpointtrue if the registration was successful, false
otherwisepublic com.google.protobuf.Message execService(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall call)
throws IOException
RegionService method using
the registered protocol handlers. Service implementations must be registered via the
Region.registerService(com.google.protobuf.Service)
method before they are available.execService in interface Regioncontroller - an RpcContoller implementation to pass to the invoked servicecall - a CoprocessorServiceCall instance identifying the service, method,
and parameters for the method invocationMessage instance containing the method's resultIOException - if no registered service handler is found or an error
occurs during the invocationRegion.registerService(com.google.protobuf.Service)protected void prepareToSplit()
public byte[] checkSplit()
public int getCompactPriority()
public RegionCoprocessorHost getCoprocessorHost()
getCoprocessorHost in interface Regionpublic void setCoprocessorHost(RegionCoprocessorHost coprocessorHost)
coprocessorHost - the new coprocessor hostpublic void startRegionOperation()
throws IOException
RegionRegion.closeRegionOperation() MUST then always be called after
the operation has completed, whether it succeeded or failed.
startRegionOperation in interface RegionIOExceptionpublic void startRegionOperation(Region.Operation op) throws IOException
RegionRegion.closeRegionOperation() MUST then always be called after
the operation has completed, whether it succeeded or failed.
startRegionOperation in interface Regionop - The operation is about to be taken on the regionIOExceptionpublic void closeRegionOperation()
throws IOException
RegioncloseRegionOperation in interface RegionIOExceptionpublic void closeRegionOperation(Region.Operation operation) throws IOException
RegionRegion.startRegionOperation(Operation)closeRegionOperation in interface RegionIOExceptionpublic static void main(String[] args) throws IOException
./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion
IOExceptionpublic long getOpenSeqNum()
getOpenSeqNum in interface Regionpublic Map<byte[],Long> getMaxStoreSeqId()
getMaxStoreSeqId in interface Regionpublic long getOldestSeqIdOfStore(byte[] familyName)
getOldestSeqIdOfStore in interface RegionRegion.getMaxFlushedSeqId() in case where we've flushed a subset of a regions column
familiespublic org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState()
getCompactionState in interface Regionpublic void reportCompactionRequestStart(boolean isMajor)
public void reportCompactionRequestEnd(boolean isMajor,
int numFiles,
long filesSizeCompacted)
public void reportCompactionRequestFailure()
public void incrementCompactionsQueuedCount()
public void decrementCompactionsQueuedCount()
public void incrementFlushesQueuedCount()
public long getSequenceId()
public void onConfigurationChange(org.apache.hadoop.conf.Configuration conf)
ConfigurationManager
object when the Configuration object is reloaded from disk.onConfigurationChange in interface ConfigurationObserverpublic void registerChildren(ConfigurationManager manager)
registerChildren in interface PropagatingConfigurationObservermanager - : to register topublic void deregisterChildren(ConfigurationManager manager)
deregisterChildren in interface PropagatingConfigurationObservermanager - : to deregister frompublic RegionSplitPolicy getSplitPolicy()
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.