@InterfaceAudience.Private public class HRegion extends Object implements HeapSize, PropagatingConfigurationObserver, Region
An Region is defined by its table and its key extent.
Locking at the Region level serves only one purpose: preventing the region from being closed (and consequently split) while other operations are ongoing. Each row level operation obtains both a row lock and a region read lock for the duration of the operation. While a scanner is being constructed, getScanner holds a read lock. If the scanner is successfully constructed, it holds a read lock until it is closed. A close takes out a write lock and consequently will block for ongoing operations and will block new operations from starting while the close is in progress.
| Modifier and Type | Class and Description | 
|---|---|
| private static class  | HRegion.BatchOperation<T>Class that tracks the progress of a batch operations, accumulating status codes and tracking
 the index at which processing is proceeding. | 
| static interface  | HRegion.BulkLoadListenerListener class to enable callers of
 bulkLoadHFile() to perform any necessary
 pre/post processing of a given bulkload call | 
| static interface  | HRegion.FlushResult | 
| static class  | HRegion.FlushResultImplObjects from this class are created when flushing to describe all the different states that
 that method ends up in. | 
| (package private) static class  | HRegion.MutationBatchOperationBatch of mutation operations. | 
| (package private) static class  | HRegion.ObservedExceptionsInBatchA class that tracks exceptions that have been observed in one batch. | 
| (package private) static class  | HRegion.PrepareFlushResultA result object from prepare flush cache stage | 
| (package private) class  | HRegion.RegionScannerImplRegionScannerImpl is used to combine scanners from multiple Stores (aka column families). | 
| (package private) static class  | HRegion.ReplayBatchOperationBatch of mutations for replay. | 
| (package private) class  | HRegion.RowLockContext | 
| static class  | HRegion.RowLockImplClass used to represent a lock on a row. | 
| (package private) static class  | HRegion.WriteState | 
Region.Operation, Region.RowLock| Constructor and Description | 
|---|
| HRegion(HRegionFileSystem fs,
       WAL wal,
       org.apache.hadoop.conf.Configuration confParam,
       TableDescriptor 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,
       RegionInfo regionInfo,
       TableDescriptor htd,
       RegionServerServices rsServices)Deprecated. 
 Use other constructors. | 
| Modifier and Type | Method and Description | 
|---|---|
| void | addRegionToSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription desc,
                   ForeignExceptionSnare exnSnare)Complete taking the snapshot on the region. | 
| Result | append(Append append)Perform one or more append operations on a row. | 
| Result | append(Append append,
      long nonceGroup,
      long nonce) | 
| private void | applyToMemStore(HStore store,
               Cell cell,
               MemStoreSizing memstoreAccounting) | 
| private void | applyToMemStore(HStore store,
               List<Cell> cells,
               boolean delta,
               MemStoreSizing memstoreAccounting) | 
| boolean | areWritesEnabled() | 
| (package private) OperationStatus[] | batchMutate(HRegion.BatchOperation<?> batchOp)Perform a batch of mutations. | 
| OperationStatus[] | batchMutate(Mutation[] mutations)Perform a batch of mutations. | 
| OperationStatus[] | batchMutate(Mutation[] mutations,
           boolean atomic,
           long nonceGroup,
           long nonce) | 
| OperationStatus[] | batchReplay(WALSplitUtil.MutationReplay[] mutations,
           long replaySeqId) | 
| void | blockUpdates() | 
| Map<byte[],List<org.apache.hadoop.fs.Path>> | bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths,
              boolean assignSeqId,
              HRegion.BulkLoadListener bulkLoadListener)Attempts to atomically load a group of hfiles. | 
| Map<byte[],List<org.apache.hadoop.fs.Path>> | bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths,
              boolean assignSeqId,
              HRegion.BulkLoadListener bulkLoadListener,
              boolean copyFile,
              List<String> clusterIds,
              boolean replicate)Attempts to atomically load a group of hfiles. | 
| boolean | checkAndMutate(byte[] row,
              byte[] family,
              byte[] qualifier,
              CompareOperator op,
              ByteArrayComparable comparator,
              TimeRange timeRange,
              Mutation mutation)Deprecated.  | 
| boolean | checkAndMutate(byte[] row,
              Filter filter,
              TimeRange timeRange,
              Mutation mutation)Deprecated.  | 
| CheckAndMutateResult | checkAndMutate(CheckAndMutate checkAndMutate)Atomically checks if a row matches the conditions and if it does, it performs the actions. | 
| boolean | checkAndRowMutate(byte[] row,
                 byte[] family,
                 byte[] qualifier,
                 CompareOperator op,
                 ByteArrayComparable comparator,
                 TimeRange timeRange,
                 RowMutations rm)Deprecated.  | 
| boolean | checkAndRowMutate(byte[] row,
                 Filter filter,
                 TimeRange timeRange,
                 RowMutations rm)Deprecated.  | 
| void | checkFamilies(Collection<byte[]> families)Check the collection of families for validity. | 
| (package private) void | checkFamily(byte[] family) | 
| (package private) void | checkInterrupt()Check thread interrupt status and throw an exception if interrupted. | 
| private void | checkMutationType(Mutation mutation) | 
| private void | checkNegativeMemStoreDataSize(long memStoreDataSize,
                             long delta) | 
| protected void | checkReadOnly() | 
| protected void | checkReadsEnabled() | 
| (package private) void | checkResources() | 
| (package private) void | checkRow(byte[] row,
        String op)Make sure this is a valid row for the HRegion | 
| private void | checkRow(Row action,
        byte[] row) | 
| Optional<byte[]> | checkSplit() | 
| Optional<byte[]> | checkSplit(boolean force)Return the split point. | 
| private void | checkTargetRegion(byte[] encodedRegionName,
                 String exceptionMsg,
                 Object payload)Checks whether the given regionName is either equal to our region, or that
 the regionName is the primary region to our corresponding range for the secondary replica. | 
| void | checkTimestamps(Map<byte[],List<Cell>> familyMap,
               long now)Check the collection of families for valid timestamps | 
| Map<byte[],List<HStoreFile>> | close()Close down this HRegion. | 
| Map<byte[],List<HStoreFile>> | close(boolean abort)Close down this HRegion. | 
| private void | closeBulkRegionOperation()Closes the lock. | 
| 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,
       HStore store,
       ThroughputController throughputController)Called by compaction thread and after region is opened to compact the
 HStores if necessary. | 
| boolean | compact(CompactionContext compaction,
       HStore store,
       ThroughputController throughputController,
       User user) | 
| (package private) void | compactStore(byte[] family,
            ThroughputController throughputController)This is a helper function that compact the given store. | 
| void | compactStores()This is a helper function that compact all the stores synchronously. | 
| static HDFSBlocksDistribution | computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf,
                             TableDescriptor tableDescriptor,
                             RegionInfo regionInfo)This is a helper function to compute HDFS block distribution on demand | 
| static HDFSBlocksDistribution | computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf,
                             TableDescriptor tableDescriptor,
                             RegionInfo regionInfo,
                             org.apache.hadoop.fs.Path tablePath)This is a helper function to compute HDFS block distribution on demand | 
| static HRegion | createHRegion(org.apache.hadoop.conf.Configuration conf,
             RegionInfo regionInfo,
             org.apache.hadoop.fs.FileSystem fs,
             org.apache.hadoop.fs.Path tableDir,
             TableDescriptor tableDesc)Create a region under the given table directory. | 
| static HRegion | createHRegion(RegionInfo info,
             org.apache.hadoop.fs.Path rootDir,
             org.apache.hadoop.conf.Configuration conf,
             TableDescriptor hTableDescriptor,
             WAL wal) | 
| static HRegion | createHRegion(RegionInfo info,
             org.apache.hadoop.fs.Path rootDir,
             org.apache.hadoop.conf.Configuration conf,
             TableDescriptor hTableDescriptor,
             WAL wal,
             boolean initialize)Convenience method creating new HRegions. | 
| static HRegion | createHRegion(RegionInfo info,
             org.apache.hadoop.fs.Path rootDir,
             org.apache.hadoop.conf.Configuration conf,
             TableDescriptor hTableDescriptor,
             WAL wal,
             boolean initialize,
             RegionServerServices rsRpcServices)Convenience method creating new HRegions. | 
| static HRegionFileSystem | createRegionDir(org.apache.hadoop.conf.Configuration configuration,
               RegionInfo ri,
               org.apache.hadoop.fs.Path rootDir)Create the region directory in the filesystem. | 
| (package private) static void | decorateRegionConfiguration(org.apache.hadoop.conf.Configuration conf)This method modifies the region's configuration in order to inject replication-related
 features | 
| void | decrementCompactionsQueuedCount() | 
| (package private) void | decrMemStoreSize(long dataSizeDelta,
                long heapSizeDelta,
                long offHeapSizeDelta,
                int cellsCountDelta) | 
| (package private) void | decrMemStoreSize(MemStoreSize mss) | 
| void | delete(Delete delete)Deletes the specified cells/row. | 
| (package private) void | delete(NavigableMap<byte[],List<Cell>> familyMap,
      Durability durability)This is used only by unit tests. | 
| private void | deleteRecoveredEdits(org.apache.hadoop.fs.FileSystem fs,
                    Iterable<org.apache.hadoop.fs.Path> files) | 
| void | deregisterChildren(ConfigurationManager manager)Needs to be called to deregister the children from the manager. | 
| protected void | disableInterrupts()If a handler thread is eligible for interrupt, make it ineligible. | 
| private void | doAbortFlushToWAL(WAL wal,
                 long flushOpSeqId,
                 Map<byte[],List<org.apache.hadoop.fs.Path>> committedFiles) | 
| private OperationStatus | doBatchMutate(Mutation mutation) | 
| private OperationStatus | doBatchMutate(Mutation mutation,
             boolean atomic) | 
| private OperationStatus | doBatchMutate(Mutation mutation,
             boolean atomic,
             long nonceGroup,
             long nonce) | 
| private Map<byte[],List<HStoreFile>> | doClose(boolean abort,
       MonitoredTask status) | 
| private MemStoreSize | doDropStoreMemStoreContentsForSeqId(HStore s,
                                   long currentSeqId) | 
| private void | doMiniBatchMutate(HRegion.BatchOperation<?> batchOp)Called to do a piece of the batch that came in to  batchMutate(Mutation[])In here we also handle replay of edits on region recover. | 
| private void | doProcessRowWithTimeout(RowProcessor<?,?> processor,
                       long now,
                       HRegion region,
                       List<Mutation> mutations,
                       WALEdit walEdit,
                       long timeout) | 
| protected void | doRegionCompactionPrep()Do preparation for pending compaction. | 
| private static void | doSyncOfUnflushedWALChanges(WAL wal,
                           RegionInfo hri)Sync unflushed WAL changes. | 
| private MultiVersionConcurrencyControl.WriteEntry | doWALAppend(WALEdit walEdit,
           Durability durability,
           List<UUID> clusterIds,
           long now,
           long nonceGroup,
           long nonce) | 
| private MultiVersionConcurrencyControl.WriteEntry | doWALAppend(WALEdit walEdit,
           Durability durability,
           List<UUID> clusterIds,
           long now,
           long nonceGroup,
           long nonce,
           long origLogSeqNum) | 
| MemStoreSize | dropMemStoreContents()Be careful, this method will drop all data in the memstore of this region. | 
| private MemStoreSize | dropMemStoreContentsForSeqId(long seqId,
                            HStore store)Drops the memstore contents after replaying a flush descriptor or region open event replay
 if the memstore edits have seqNums smaller than the given seq id | 
| private void | dropPrepareFlushIfPossible()If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult | 
| protected void | enableInterrupts()If a handler thread was made ineligible for interrupt via { disableInterrupts(),
 make it eligible again. | 
| boolean | equals(Object o) | 
| com.google.protobuf.Message | execService(com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall call)Executes a single protocol buffer coprocessor endpoint  Servicemethod using
 the registered protocol handlers. | 
| HRegion.FlushResult | flush(boolean flushAllStores)Flush the cache. | 
| HRegion.FlushResultImpl | flushcache(boolean flushAllStores,
          boolean writeFlushRequestWalMarker,
          FlushLifeCycleTracker tracker) | 
| HRegion.FlushResultImpl | flushcache(List<byte[]> families,
          boolean writeFlushRequestWalMarker,
          FlushLifeCycleTracker tracker)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) | 
| (package private) org.apache.hadoop.conf.Configuration | getBaseConf()A split takes the config from the parent region & passes it to the daughter
 region's constructor. | 
| BlockCache | getBlockCache() | 
| long | getBlockedRequestsCount() | 
| CellComparator | getCellComparator()The comparator to be used with the region | 
| long | getCheckAndMutateChecksFailed() | 
| long | getCheckAndMutateChecksPassed() | 
| CompactionState | getCompactionState() | 
| int | getCompactPriority() | 
| 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() | 
| long | getFilteredReadRequestsCount() | 
| HDFSBlocksDistribution | getHDFSBlocksDistribution() | 
| org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats | getLoadStatistics() | 
| ConcurrentHashMap<HashedBytes,HRegion.RowLockContext> | getLockedRows() | 
| long | getMaxFlushedSeqId() | 
| Map<byte[],Long> | getMaxStoreSeqId() | 
| long | getMemStoreDataSize() | 
| long | getMemStoreFlushSize() | 
| long | getMemStoreHeapSize() | 
| long | getMemStoreOffHeapSize() | 
| MetricsRegion | getMetrics() | 
| MobFileCache | getMobFileCache() | 
| 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) | 
| (package private) static ThreadPoolExecutor | getOpenAndCloseThreadPool(int maxThreads,
                         String threadNamePrefix) | 
| long | getOpenSeqNum() | 
| (package private) HRegion.PrepareFlushResult | getPrepareFlushResult() | 
| int | getReadLockCount() | 
| long | getReadPoint() | 
| long | getReadPoint(IsolationLevel isolationLevel) | 
| long | getReadRequestsCount() | 
| static org.apache.hadoop.fs.Path | getRegionDir(org.apache.hadoop.fs.Path tabledir,
            String name)Deprecated. 
 For tests only; to be removed. | 
| HRegionFileSystem | getRegionFileSystem() | 
| RegionInfo | getRegionInfo() | 
| (package private) RegionServerServices | getRegionServerServices() | 
| RegionServicesForStores | getRegionServicesForStores() | 
| (package private) HRegionWALFileSystem | getRegionWALFileSystem() | 
| NavigableMap<byte[],Integer> | getReplicationScope() | 
| 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. | 
| protected Region.RowLock | getRowLockInternal(byte[] row,
                  boolean readLock,
                  Region.RowLock prevRowLock) | 
| HRegion.RegionScannerImpl | getScanner(Scan scan)Return an iterator that scans over the HRegion, returning the indicated
 columns and rows specified by the  Scan. | 
| HRegion.RegionScannerImpl | getScanner(Scan scan,
          List<KeyValueScanner> additionalScanners)Return an iterator that scans over the HRegion, returning the indicated columns and rows
 specified by the  Scan. | 
| private HRegion.RegionScannerImpl | getScanner(Scan scan,
          List<KeyValueScanner> additionalScanners,
          long nonceGroup,
          long nonce) | 
| long | getSmallestReadPoint() | 
| private Collection<HStore> | getSpecificStores(List<byte[]> families)get stores which matches the specified families | 
| (package private) RegionSplitPolicy | getSplitPolicy() | 
| HStore | getStore(byte[] column)Return the Store for the given family | 
| private HStore | getStore(Cell cell)Return HStore instance. | 
| List<String> | getStoreFileList(byte[][] columns) | 
| protected ThreadPoolExecutor | getStoreFileOpenAndCloseThreadPool(String threadNamePrefix) | 
| private NavigableMap<byte[],List<org.apache.hadoop.fs.Path>> | getStoreFiles() | 
| protected ThreadPoolExecutor | getStoreOpenAndCloseThreadPool(String threadNamePrefix) | 
| List<HStore> | getStores()Return the list of Stores managed by this region | 
| TableDescriptor | getTableDescriptor() | 
| WAL | getWAL() | 
| (package private) org.apache.hadoop.fs.FileSystem | getWalFileSystem() | 
| org.apache.hadoop.fs.Path | getWALRegionDir() | 
| long | getWriteRequestsCount() | 
| private void | handleException(org.apache.hadoop.fs.FileSystem fs,
               org.apache.hadoop.fs.Path edits,
               IOException e) | 
| int | hashCode() | 
| private static boolean | hasMultipleColumnFamilies(Collection<Pair<byte[],String>> familyPaths)Determines whether multiple column families are present
 Precondition: familyPaths is not null | 
| boolean | hasReferences() | 
| long | heapSize() | 
| (package private) void | incMemStoreSize(long dataSizeDelta,
               long heapSizeDelta,
               long offHeapSizeDelta,
               int cellsCountDelta) | 
| (package private) void | incMemStoreSize(MemStoreSize mss)Increase the size of mem store in this region and the size of global mem
 store | 
| Result | increment(Increment increment)Perform one or more increment operations on a row. | 
| Result | increment(Increment increment,
         long nonceGroup,
         long nonce) | 
| void | incrementCompactionsQueuedCount() | 
| void | incrementFlushesQueuedCount() | 
| long | initialize()Deprecated. 
 use HRegion.createHRegion() or HRegion.openHRegion() | 
| (package private) long | initialize(CancelableProgressable reporter)Initialize this region. | 
| private long | initializeRegionInternals(CancelableProgressable reporter,
                         MonitoredTask status) | 
| private long | initializeStores(CancelableProgressable reporter,
                MonitoredTask status)Open all Stores. | 
| private long | initializeStores(CancelableProgressable reporter,
                MonitoredTask status,
                boolean warmup) | 
| private void | initializeWarmup(CancelableProgressable reporter) | 
| protected HStore | instantiateHStore(ColumnFamilyDescriptor family,
                 boolean warmup) | 
| protected RegionScanner | instantiateRegionScanner(Scan scan,
                        List<KeyValueScanner> additionalScanners) | 
| protected HRegion.RegionScannerImpl | instantiateRegionScanner(Scan scan,
                        List<KeyValueScanner> additionalScanners,
                        long nonceGroup,
                        long nonce) | 
| private HRegion.FlushResultImpl | internalFlushcache(Collection<HStore> storesToFlush,
                  MonitoredTask status,
                  boolean writeFlushWalMarker,
                  FlushLifeCycleTracker tracker)Flushing given stores. | 
| private HRegion.FlushResult | internalFlushcache(MonitoredTask status)Flushing all stores. | 
| protected HRegion.FlushResultImpl | internalFlushcache(WAL wal,
                  long myseqid,
                  Collection<HStore> storesToFlush,
                  MonitoredTask status,
                  boolean writeFlushWalMarker,
                  FlushLifeCycleTracker tracker)Flush the memstore. | 
| protected HRegion.FlushResultImpl | internalFlushCacheAndCommit(WAL wal,
                           MonitoredTask status,
                           HRegion.PrepareFlushResult prepareResult,
                           Collection<HStore> storesToFlush) | 
| protected HRegion.PrepareFlushResult | internalPrepareFlushCache(WAL wal,
                         long myseqid,
                         Collection<HStore> storesToFlush,
                         MonitoredTask status,
                         boolean writeFlushWalMarker,
                         FlushLifeCycleTracker tracker) | 
| private void | interruptRegionOperations()Interrupt any region options that have acquired the region lock via
  startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation),
 orstartBulkRegionOperation(boolean). | 
| private boolean | isAllFamilies(Collection<HStore> families) | 
| boolean | isAvailable() | 
| boolean | isClosed() | 
| boolean | isClosing() | 
| private boolean | isFlushSize(MemStoreSize size) | 
| boolean | isLoadingCfsOnDemandDefault() | 
| boolean | isMergeable() | 
| boolean | isReadOnly() | 
| boolean | isSplittable() | 
| private static boolean | isZeroLengthThenDelete(org.apache.hadoop.fs.FileSystem fs,
                      org.apache.hadoop.fs.FileStatus stat,
                      org.apache.hadoop.fs.Path p)make sure have been through lease recovery before get file status, so the file length can be
 trusted. | 
| private long | loadRecoveredHFilesIfAny(Collection<HStore> stores) | 
| private void | lock(Lock lock) | 
| private void | lock(Lock lock,
    int multiplier)Try to acquire a lock. | 
| private void | logFatLineOnFlush(Collection<HStore> storesToFlush,
                 long sequenceId)Utility method broken out of internalPrepareFlushCache so that method is smaller. | 
| private void | logRegionFiles() | 
| private boolean | matches(CompareOperator op,
       int compareResult) | 
| (package private) void | metricsUpdateForGet(List<Cell> results,
                   long before) | 
| Result | mutateRow(RowMutations rm)Performs multiple mutations atomically on a single row. | 
| void | mutateRowsWithLocks(Collection<Mutation> mutations,
                   Collection<byte[]> rowsToLock,
                   long nonceGroup,
                   long nonce)Perform atomic (all or none) mutations within the region. | 
| static HRegion | newHRegion(org.apache.hadoop.fs.Path tableDir,
          WAL wal,
          org.apache.hadoop.fs.FileSystem fs,
          org.apache.hadoop.conf.Configuration conf,
          RegionInfo regionInfo,
          TableDescriptor htd,
          RegionServerServices rsServices)A utility method to create new instances of HRegion based on the  HConstants.REGION_IMPLconfiguration property. | 
| void | onConfigurationChange(org.apache.hadoop.conf.Configuration conf)This method would be called by the  ConfigurationManagerobject when theConfigurationobject 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,
           RegionInfo info,
           TableDescriptor 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,
           RegionInfo info,
           TableDescriptor 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(org.apache.hadoop.fs.Path rootDir,
           RegionInfo info,
           TableDescriptor htd,
           WAL wal,
           org.apache.hadoop.conf.Configuration conf)Open a Region. | 
| static HRegion | openHRegion(org.apache.hadoop.fs.Path rootDir,
           RegionInfo info,
           TableDescriptor 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 | openHRegion(RegionInfo info,
           TableDescriptor htd,
           WAL wal,
           org.apache.hadoop.conf.Configuration conf)Open a Region. | 
| static HRegion | openHRegion(RegionInfo info,
           TableDescriptor htd,
           WAL wal,
           org.apache.hadoop.conf.Configuration conf,
           RegionServerServices rsServices,
           CancelableProgressable reporter)Open a Region. | 
| static HRegion | openHRegionFromTableDir(org.apache.hadoop.conf.Configuration conf,
                       org.apache.hadoop.fs.FileSystem fs,
                       org.apache.hadoop.fs.Path tableDir,
                       RegionInfo info,
                       TableDescriptor htd,
                       WAL wal,
                       RegionServerServices rsServices,
                       CancelableProgressable reporter)Open a Region. | 
| static HRegion | openReadOnlyFileSystemHRegion(org.apache.hadoop.conf.Configuration conf,
                             org.apache.hadoop.fs.FileSystem fs,
                             org.apache.hadoop.fs.Path tableDir,
                             RegionInfo info,
                             TableDescriptor 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. | 
| (package private) void | prepareGet(Get get) | 
| private void | preProcess(RowProcessor<?,?> processor,
          WALEdit walEdit) | 
| 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. | 
| (package private) void | put(byte[] row,
   byte[] family,
   List<Cell> edits)Add updates first to the wal and then add values to memstore. | 
| void | put(Put put)Puts some data in the table. | 
| private void | recordMutationWithoutWal(Map<byte[],List<Cell>> familyMap)Update LongAdders for number of puts without wal and the size of possible data loss. | 
| 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. | 
| protected boolean | refreshStoreFiles(boolean force) | 
| 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  Servicesubclass as a coprocessor endpoint to
 be available for handling Region#execService(com.google.protobuf.RpcController,
    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall) calls. | 
| private void | releaseRowLocks(List<Region.RowLock> rowLocks) | 
| private void | replayFlushInStores(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush,
                   HRegion.PrepareFlushResult prepareFlushResult,
                   boolean dropMemstoreSnapshot)Replays the given flush descriptor by opening the flush files in stores and dropping the
 memstore snapshots if requested. | 
| private long | replayRecoveredEdits(org.apache.hadoop.fs.Path edits,
                    Map<byte[],Long> maxSeqIdInStores,
                    CancelableProgressable reporter,
                    org.apache.hadoop.fs.FileSystem fs) | 
| private long | replayRecoveredEditsForPaths(long minSeqIdForTheRegion,
                            org.apache.hadoop.fs.FileSystem fs,
                            NavigableSet<org.apache.hadoop.fs.Path> files,
                            CancelableProgressable reporter,
                            org.apache.hadoop.fs.Path regionDir) | 
| (package private) long | replayRecoveredEditsIfAny(Map<byte[],Long> maxSeqIdInStores,
                         CancelableProgressable reporter,
                         MonitoredTask status)Read the edits put under this region by wal splitting process. | 
| (package private) void | replayWALBulkLoadEventMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor bulkLoadEvent) | 
| (package private) void | replayWALCompactionMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor compaction,
                         boolean pickCompactionFiles,
                         boolean removeFiles,
                         long replaySeqId)Call to complete a compaction. | 
| private void | replayWALFlushAbortMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush) | 
| private void | replayWALFlushCannotFlushMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush,
                               long replaySeqId) | 
| (package private) void | replayWALFlushCommitMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush) | 
| (package private) void | replayWALFlushMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush,
                    long replaySeqId) | 
| (package private) HRegion.PrepareFlushResult | replayWALFlushStartMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush)Replay the flush marker from primary region by creating a corresponding snapshot of
 the store memstores, only if the memstores do not have a higher seqId from an earlier wal
 edit (because the events may be coming out of order). | 
| (package private) void | replayWALRegionEventMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor regionEvent) | 
| void | reportCompactionRequestEnd(boolean isMajor,
                          int numFiles,
                          long filesSizeCompacted) | 
| void | reportCompactionRequestFailure() | 
| void | reportCompactionRequestStart(boolean isMajor) | 
| void | requestCompaction(byte[] family,
                 String why,
                 int priority,
                 boolean major,
                 CompactionLifeCycleTracker tracker)Request compaction for the given family | 
| void | requestCompaction(String why,
                 int priority,
                 boolean major,
                 CompactionLifeCycleTracker tracker)Request compaction on this region. | 
| private void | requestFlush() | 
| void | requestFlush(FlushLifeCycleTracker tracker)Request flush on this region. | 
| private void | requestFlush0(FlushLifeCycleTracker tracker) | 
| private void | requestFlushIfNeeded() | 
| protected void | restoreEdit(HStore s,
           Cell cell,
           MemStoreSizing memstoreAccounting)Used by tests | 
| (package private) void | rewriteCellTags(Map<byte[],List<Cell>> familyMap,
               Mutation m)Possibly rewrite incoming cell tags. | 
| static boolean | rowIsInRange(RegionInfo info,
            byte[] row)Determines if the specified row is within the row range specified by the
 specified RegionInfo | 
| static boolean | rowIsInRange(RegionInfo info,
            byte[] row,
            int offset,
            short length) | 
| void | setBlockCache(BlockCache blockCache)Only used for unit test which doesn't start region server. | 
| void | setClosing(boolean closing)Exposed for some very specific unit tests. | 
| (package private) org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder | setCompleteSequenceId(org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder regionLoadBldr) | 
| void | setCoprocessorHost(RegionCoprocessorHost coprocessorHost) | 
| (package private) void | setHTableSpecificConf() | 
| void | setMobFileCache(MobFileCache mobFileCache)Only used for unit test which doesn't start region server. | 
| void | setReadRequestsCount(long readRequestsCount) | 
| void | setReadsEnabled(boolean readsEnabled) | 
| void | setRestoredRegion(boolean restoredRegion) | 
| (package private) void | setTableDescriptor(TableDescriptor desc) | 
| void | setTimeoutForWriteLock(long timeoutForWriteLock)The  doClose(boolean, org.apache.hadoop.hbase.monitoring.MonitoredTask)will block forever if someone tries proving the dead lock via the unit test. | 
| void | setWriteRequestsCount(long writeRequestsCount) | 
| (package private) boolean | shouldFlush(StringBuilder whyFlush)Should the memstore be flushed now | 
| (package private) boolean | shouldFlushStore(HStore store)Should the store be flushed because it is old enough. | 
| private boolean | shouldSyncWAL()Check whether we should sync the wal from the table's durability settings | 
| private static List<Cell> | sort(List<Cell> cells,
    CellComparator comparator) | 
| private void | startBulkRegionOperation(boolean writeLockNeeded)This method needs to be called before any public call that reads or
 modifies stores in bulk. | 
| 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. | 
| private void | sync(long txid,
    Durability durability)Calls sync with the given transaction ID | 
| (package private) void | throwException(String title,
              String regionName) | 
| (package private) IOException | throwOnInterrupt(Throwable t)Throw the correct exception upon interrupt | 
| String | toString() | 
| void | unblockUpdates() | 
| private static void | updateCellTimestamps(Iterable<List<Cell>> cellItr,
                    byte[] now)Replace any cell timestamps set to  HConstants.LATEST_TIMESTAMPprovided current timestamp. | 
| (package private) void | updateDeleteLatestVersionTimestamp(Cell cell,
                                  Get get,
                                  int count,
                                  byte[] byteNow) | 
| private void | updateSequenceId(Iterable<List<Cell>> cellItr,
                long sequenceId) | 
| void | waitForFlushes()Wait for all current flushes of the region to complete | 
| boolean | waitForFlushes(long timeout)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(RegionInfo info,
             TableDescriptor htd,
             WAL wal,
             org.apache.hadoop.conf.Configuration conf,
             RegionServerServices rsServices,
             CancelableProgressable reporter) | 
| private boolean | worthPreFlushing() | 
| private boolean | writeFlushRequestMarkerToWAL(WAL wal,
                            boolean writeFlushWalMarker)Writes a marker to WAL indicating a flush is requested but cannot be complete due to various
 reasons. | 
| private void | writeRegionCloseMarker(WAL wal) | 
| protected void | writeRegionOpenMarker(WAL wal,
                     long openSeqId) | 
clone, finalize, getClass, notify, notifyAll, wait, wait, waitcheckAndMutate, checkAndMutate, checkAndRowMutate, checkAndRowMutateprivate static final org.slf4j.Logger LOG
public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY
public static final String HBASE_MAX_CELL_SIZE_KEY
public static final int DEFAULT_MAX_CELL_SIZE
public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE
public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE
public static final String WAL_HSYNC_CONF_KEY
public static final boolean DEFAULT_WAL_HSYNC
public static final String SPECIAL_RECOVERED_EDITS_DIR
public static final String USE_META_CELL_COMPARATOR
MetaCellComparator even if we are not meta region. Used when creating
 master local region.public static final boolean DEFAULT_USE_META_CELL_COMPARATOR
final AtomicBoolean closed
final AtomicBoolean closing
private volatile long maxFlushedSeqId
private volatile long lastFlushOpSeqId
maxFlushedSeqId when flushing a single column family. In this case,
 maxFlushedSeqId will be older than the oldest edit in memory.protected volatile long lastReplayedOpenRegionSeqId
protected volatile long lastReplayedCompactionSeqId
private final ConcurrentHashMap<HashedBytes,HRegion.RowLockContext> lockedRows
private Map<String,com.google.protobuf.Service> coprocessorServiceHandlers
private final MemStoreSizing memStoreSizing
RegionServicesForStores regionServicesForStores
final LongAdder numMutationsWithoutWAL
final LongAdder dataInMemoryWithoutWAL
final LongAdder checkAndMutateChecksPassed
final LongAdder checkAndMutateChecksFailed
final LongAdder readRequestsCount
final LongAdder filteredReadRequestsCount
final LongAdder writeRequestsCount
private final LongAdder blockedRequestsCount
final LongAdder compactionsFinished
final LongAdder compactionsFailed
final LongAdder compactionNumFilesCompacted
final LongAdder compactionNumBytesCompacted
final LongAdder compactionsQueued
final LongAdder flushesQueued
private BlockCache blockCache
private MobFileCache mobFileCache
private final HRegionFileSystem fs
protected final org.apache.hadoop.conf.Configuration conf
private final org.apache.hadoop.conf.Configuration baseConf
private final int rowLockWaitDuration
static final int DEFAULT_ROWLOCK_WAIT_DURATION
private org.apache.hadoop.fs.Path regionDir
private org.apache.hadoop.fs.FileSystem walFS
private boolean isRestoredRegion
final long busyWaitDuration
static final long DEFAULT_BUSY_WAIT_DURATION
final int maxBusyWaitMultiplier
final long maxBusyWaitDuration
final long maxCellSize
private final int miniBatchSize
static final long DEFAULT_ROW_PROCESSOR_TIMEOUT
final ExecutorService rowProcessorExecutor
private final ConcurrentHashMap<RegionScanner,Long> scannerReadPoints
private long openSeqNum
private boolean isLoadingCfsOnDemandDefault
private final AtomicInteger majorInProgress
private final AtomicInteger minorInProgress
Map<byte[],Long> maxSeqIdInStores
private HRegion.PrepareFlushResult prepareFlushResult
private volatile ConfigurationManager configurationManager
private volatile Long timeoutForWriteLock
private final CellComparator cellComparator
final HRegion.WriteState writestate
long memstoreFlushSize
final long timestampSlop
final long rowProcessorTimeout
private final ConcurrentMap<HStore,Long> lastStoreFlushTimeMap
protected RegionServerServices rsServices
private RegionServerAccounting rsAccounting
private long flushCheckInterval
private long flushPerChanges
private long blockingMemStoreSize
final ReentrantReadWriteLock lock
final ConcurrentHashMap<Thread,Boolean> regionLockHolders
private final ReentrantReadWriteLock updatesLock
private final MultiVersionConcurrencyControl mvcc
private RegionCoprocessorHost coprocessorHost
private TableDescriptor htableDescriptor
private RegionSplitPolicy splitPolicy
private FlushPolicy flushPolicy
private final MetricsRegion metricsRegion
private final MetricsRegionWrapperImpl metricsRegionWrapper
private final Durability regionDurability
private final boolean regionStatsEnabled
private final NavigableMap<byte[],Integer> replicationScope
private final StoreHotnessProtector storeHotnessProtector
public static final String FAIR_REENTRANT_CLOSE_LOCK
public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK
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 String CLOSE_WAIT_ABORT
public static final boolean DEFAULT_CLOSE_WAIT_ABORT
public static final String CLOSE_WAIT_TIME
public static final long DEFAULT_CLOSE_WAIT_TIME
public static final String CLOSE_WAIT_INTERVAL
public static final long DEFAULT_CLOSE_WAIT_INTERVAL
private static final byte[] FOR_UNIT_TESTS_ONLY
public static final long FIXED_OVERHEAD
public static final long DEEP_OVERHEAD
private static final List<Cell> MOCKED_LIST
@Deprecated public HRegion(org.apache.hadoop.fs.Path tableDir, WAL wal, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration confParam, RegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices)
createHRegion(org.apache.hadoop.hbase.client.RegionInfo, org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, org.apache.hadoop.hbase.client.TableDescriptor, org.apache.hadoop.hbase.wal.WAL, boolean) or openHRegion(org.apache.hadoop.hbase.client.RegionInfo, org.apache.hadoop.hbase.client.TableDescriptor, 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 - - RegionInfo 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, TableDescriptor htd, RegionServerServices rsServices)
createHRegion(org.apache.hadoop.hbase.client.RegionInfo, org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration, org.apache.hadoop.hbase.client.TableDescriptor, org.apache.hadoop.hbase.wal.WAL, boolean) or openHRegion(org.apache.hadoop.hbase.client.RegionInfo, org.apache.hadoop.hbase.client.TableDescriptor, 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 void setRestoredRegion(boolean restoredRegion)
public long getSmallestReadPoint()
void setHTableSpecificConf()
@Deprecated public long initialize() throws IOException
IOException - elong initialize(CancelableProgressable reporter) throws IOException
reporter - Tickle every so often if initialize is taking a while.IOException - eprivate long initializeRegionInternals(CancelableProgressable reporter, MonitoredTask status) throws IOException
IOExceptionprivate long initializeStores(CancelableProgressable reporter, MonitoredTask status) throws IOException
reporter - status - IOExceptionprivate long initializeStores(CancelableProgressable reporter, MonitoredTask status, boolean warmup) throws IOException
IOExceptionprivate void initializeWarmup(CancelableProgressable reporter) throws IOException
IOExceptionprivate NavigableMap<byte[],List<org.apache.hadoop.fs.Path>> getStoreFiles()
protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException
IOExceptionprivate void writeRegionCloseMarker(WAL wal) throws IOException
IOExceptionpublic boolean hasReferences()
public void blockUpdates()
public void unblockUpdates()
public HDFSBlocksDistribution getHDFSBlocksDistribution()
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf, TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException
conf - configurationtableDescriptor - TableDescriptor of the tableregionInfo - encoded name of the regionIOExceptionpublic static HDFSBlocksDistribution computeHDFSBlocksDistribution(org.apache.hadoop.conf.Configuration conf, TableDescriptor tableDescriptor, RegionInfo regionInfo, org.apache.hadoop.fs.Path tablePath) throws IOException
conf - configurationtableDescriptor - TableDescriptor of the tableregionInfo - encoded name of the regiontablePath - the table directoryIOExceptionvoid incMemStoreSize(MemStoreSize mss)
void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta, int cellsCountDelta)
void decrMemStoreSize(MemStoreSize mss)
void decrMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta, int cellsCountDelta)
private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta)
public RegionInfo getRegionInfo()
getRegionInfo in interface RegionRegionServerServices getRegionServerServices()
RegionServerServices used by this HRegion.
 Can be null.public long getReadRequestsCount()
getReadRequestsCount in interface Regionpublic long getFilteredReadRequestsCount()
getFilteredReadRequestsCount in interface Regionpublic long getWriteRequestsCount()
getWriteRequestsCount in interface Regionpublic long getMemStoreDataSize()
getMemStoreDataSize in interface Regionpublic long getMemStoreHeapSize()
getMemStoreHeapSize in interface Regionpublic long getMemStoreOffHeapSize()
getMemStoreOffHeapSize in interface Regionpublic RegionServicesForStores getRegionServicesForStores()
public 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()
public boolean isClosed()
public boolean isClosing()
public boolean isReadOnly()
isReadOnly in interface Regionpublic boolean isAvailable()
isAvailable in interface Regionpublic boolean isSplittable()
isSplittable in interface Regionpublic boolean isMergeable()
isMergeable in interface Regionpublic boolean areWritesEnabled()
public MultiVersionConcurrencyControl getMVCC()
public long getMaxFlushedSeqId()
getMaxFlushedSeqId in interface Regionpublic long getReadPoint(IsolationLevel isolationLevel)
null for defaultpublic boolean isLoadingCfsOnDemandDefault()
public Map<byte[],List<HStoreFile>> 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<HStoreFile>> 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 setTimeoutForWriteLock(long timeoutForWriteLock)
doClose(boolean, org.apache.hadoop.hbase.monitoring.MonitoredTask) will block forever if someone tries proving the dead lock via the unit test.
 Instead of blocking, the doClose(boolean, org.apache.hadoop.hbase.monitoring.MonitoredTask) will throw exception if you set the timeout.timeoutForWriteLock - the second time to wait for the write lock in doClose(boolean, org.apache.hadoop.hbase.monitoring.MonitoredTask)private Map<byte[],List<HStoreFile>> doClose(boolean abort, MonitoredTask status) throws IOException
IOExceptionpublic void waitForFlushesAndCompactions()
public void waitForFlushes()
public boolean waitForFlushes(long timeout)
RegionwaitForFlushes in interface Regiontimeout - The maximum time to wait in milliseconds.protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(String threadNamePrefix)
protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(String threadNamePrefix)
static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads, String threadNamePrefix)
private boolean worthPreFlushing()
public TableDescriptor getTableDescriptor()
getTableDescriptor in interface Regionvoid setTableDescriptor(TableDescriptor desc)
public BlockCache getBlockCache()
public void setBlockCache(BlockCache blockCache)
public MobFileCache getMobFileCache()
public void setMobFileCache(MobFileCache mobFileCache)
RegionSplitPolicy getSplitPolicy()
org.apache.hadoop.conf.Configuration getBaseConf()
public org.apache.hadoop.fs.FileSystem getFilesystem()
FileSystem being used by this regionpublic HRegionFileSystem getRegionFileSystem()
HRegionFileSystem used by this regionHRegionWALFileSystem getRegionWALFileSystem() throws IOException
HRegionFileSystem used by this regionIOExceptionorg.apache.hadoop.fs.FileSystem getWalFileSystem() throws IOException
FileSystem being used by this regionIOExceptionpublic 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 compactionIOExceptionorg.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder setCompleteSequenceId(org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder regionLoadBldr)
protected void doRegionCompactionPrep() throws IOException
IOExceptionpublic void compact(boolean majorCompaction) throws IOException
This 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.
majorCompaction - True to force a major compaction regardless of thresholdsIOExceptionpublic void compactStores() throws IOException
It is used by utilities and testing
IOExceptionvoid compactStore(byte[] family, ThroughputController throughputController) throws IOException
It is used by utilities and testing
IOExceptionpublic boolean compact(CompactionContext compaction, HStore store, ThroughputController throughputController) throws IOException
This operation could block for a long time, so don't call it from a time-sensitive thread. Note that no locking is necessary at this level because compaction only conflicts with a region split, and that cannot happen because the region server does them sequentially and not in parallel.
compaction - Compaction details, obtained by requestCompaction()throughputController - IOExceptionpublic boolean compact(CompactionContext compaction, HStore store, ThroughputController throughputController, User user) throws IOException
IOExceptionpublic HRegion.FlushResult flush(boolean flushAllStores) throws IOException
When 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.
flushAllStores - whether we want to force a flush of all storesIOException - general io exceptions
 because a snapshot was not properly persisted.public HRegion.FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException
IOExceptionpublic HRegion.FlushResultImpl flushcache(List<byte[]> families, boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException
This method may block for some time, so it should not be called from a time-sensitive thread.
families - stores of region to flush.writeFlushRequestWalMarker - whether to write the flush request marker to WALtracker - used to track the life cycle of this flushIOException - 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.private Collection<HStore> getSpecificStores(List<byte[]> families)
boolean shouldFlushStore(HStore store)
Every FlushPolicy should call this to determine whether a store is old enough to flush (except that you always flush all stores). Otherwise the method will always returns true which will make a lot of flush requests.
boolean shouldFlush(StringBuilder whyFlush)
private HRegion.FlushResult internalFlushcache(MonitoredTask status) throws IOException
private HRegion.FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException
protected HRegion.FlushResultImpl internalFlushcache(WAL wal, long myseqid, Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) 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.protected HRegion.PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid, Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException
IOExceptionprivate void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId)
private void doAbortFlushToWAL(WAL wal, long flushOpSeqId, Map<byte[],List<org.apache.hadoop.fs.Path>> committedFiles)
private static void doSyncOfUnflushedWALChanges(WAL wal, RegionInfo hri) throws IOException
IOExceptionprivate boolean isAllFamilies(Collection<HStore> families)
private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker)
wal - protected HRegion.FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status, HRegion.PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException
IOExceptionprotected long getNextSequenceId(WAL wal) throws IOException
IOExceptionpublic HRegion.RegionScannerImpl getScanner(Scan scan) throws IOException
RegionScan.
 This Iterator must be closed by the caller.
getScanner in interface Regionscan - configured ScanIOException - read exceptionspublic HRegion.RegionScannerImpl 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 exceptionsprivate HRegion.RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException
IOExceptionprotected RegionScanner instantiateRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException
IOExceptionprotected HRegion.RegionScannerImpl instantiateRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException
IOExceptionpublic void prepareDelete(Delete delete) throws IOException
delete - The passed delete is modified by this method. WARNING!IOExceptionpublic void delete(Delete delete) throws IOException
Regiondelete in interface RegionIOExceptionvoid delete(NavigableMap<byte[],List<Cell>> familyMap, Durability durability) throws IOException
familyMap - map of family to edits for the given family.IOExceptionpublic void prepareDeleteTimestamps(Mutation mutation, Map<byte[],List<Cell>> familyMap, byte[] byteNow) throws IOException
Caller should have the row and region locks.
mutation - familyMap - byteNow - IOExceptionvoid updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow) throws IOException
IOExceptionpublic void put(Put put) throws IOException
Regionput in interface RegionIOExceptionpublic OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup, long nonce) throws IOException
IOExceptionpublic OperationStatus[] batchMutate(Mutation[] mutations) throws IOException
RegionPlease 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.
batchMutate in interface Regionmutations - the list of mutationsIOExceptionpublic OperationStatus[] batchReplay(WALSplitUtil.MutationReplay[] mutations, long replaySeqId) throws IOException
IOExceptionOperationStatus[] batchMutate(HRegion.BatchOperation<?> batchOp) throws IOException
Durability.SKIP_WAL.
 This function is called from batchReplay(WALSplitUtil.MutationReplay[], long) with
 HRegion.ReplayBatchOperation instance and batchMutate(Mutation[]) with
 HRegion.MutationBatchOperation instance as an argument. As the processing of replay batch
 and mutation batch is very similar, lot of code is shared by providing generic methods in
 base class HRegion.BatchOperation. The logic for this method and
 doMiniBatchMutate(BatchOperation) is implemented using methods in base class which
 are overridden by derived classes to implement special behavior.
batchOp - contains the list of mutationsIOException - if an IO problem is encounteredprivate void doMiniBatchMutate(HRegion.BatchOperation<?> batchOp) throws IOException
batchMutate(Mutation[])
 In here we also handle replay of edits on region recover. Also gets change in size brought
 about by applying batchOp.IOExceptionprotected Durability getEffectiveDurability(Durability d)
@Deprecated public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException
RegioncheckAndMutate in interface Regionrow - to checkfamily - column family to checkqualifier - column qualifier to checkop - the comparison operatorcomparator - the expected valuetimeRange - time range to checkmutation - data to put if check succeedsIOException@Deprecated public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) throws IOException
RegioncheckAndMutate in interface Regionrow - to checkfilter - the filtertimeRange - time range to checkmutation - data to put if check succeedsIOException@Deprecated public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException
RegioncheckAndRowMutate in interface Regionrow - to checkfamily - column family to checkqualifier - column qualifier to checkop - the comparison operatorcomparator - the expected valuetimeRange - time range to checkrm - data to put if check succeedsIOException@Deprecated public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm) throws IOException
RegioncheckAndRowMutate in interface Regionrow - to checkfilter - the filtertimeRange - time range to checkrm - data to put if check succeedsIOExceptionpublic CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException
RegioncheckAndMutate in interface RegioncheckAndMutate - the CheckAndMutate objectIOException - if an error occurred in this methodprivate void checkMutationType(Mutation mutation) throws DoNotRetryIOException
DoNotRetryIOExceptionprivate void checkRow(Row action, byte[] row) throws DoNotRetryIOException
DoNotRetryIOExceptionprivate boolean matches(CompareOperator op, int compareResult)
private OperationStatus doBatchMutate(Mutation mutation) throws IOException
IOExceptionprivate OperationStatus doBatchMutate(Mutation mutation, boolean atomic) throws IOException
IOExceptionprivate OperationStatus doBatchMutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce) throws IOException
IOExceptionpublic void addRegionToSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.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 failprivate void updateSequenceId(Iterable<List<Cell>> cellItr, long sequenceId) throws IOException
IOExceptionprivate static void updateCellTimestamps(Iterable<List<Cell>> cellItr, byte[] now) throws IOException
HConstants.LATEST_TIMESTAMP
 provided current timestamp.cellItr - now - IOExceptionvoid rewriteCellTags(Map<byte[],List<Cell>> familyMap, Mutation m)
void checkResources() throws RegionTooBusyException
RegionTooBusyExceptionprotected void checkReadOnly() throws IOException
IOException - Throws exception if region is in read-only mode.protected void checkReadsEnabled() throws IOException
IOExceptionpublic void setReadsEnabled(boolean readsEnabled)
void put(byte[] row, byte[] family, List<Cell> edits) throws IOException
edits - Cell updates by columnIOExceptionprivate void applyToMemStore(HStore store, List<Cell> cells, boolean delta, MemStoreSizing memstoreAccounting) throws IOException
delta - If we are doing delta changes -- e.g. increment/append -- then this flag will be
          set; when set we will run operations that make sense in the increment/append scenario
          but that do not make sense otherwise.IOExceptionapplyToMemStore(HStore, Cell, MemStoreSizing)private void applyToMemStore(HStore store, Cell cell, MemStoreSizing memstoreAccounting) throws IOException
IOExceptionapplyToMemStore(HStore, List, boolean, MemStoreSizing)public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException
families - NoSuchColumnFamilyExceptionpublic void checkTimestamps(Map<byte[],List<Cell>> familyMap, long now) throws FailedSanityCheckException
familyMap - now - current timestampFailedSanityCheckExceptionprivate boolean isFlushSize(MemStoreSize size)
private void deleteRecoveredEdits(org.apache.hadoop.fs.FileSystem fs, Iterable<org.apache.hadoop.fs.Path> files) throws IOException
IOExceptionlong 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.IOExceptionprivate long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, org.apache.hadoop.fs.FileSystem fs, NavigableSet<org.apache.hadoop.fs.Path> files, CancelableProgressable reporter, org.apache.hadoop.fs.Path regionDir) throws IOException
IOExceptionprivate void handleException(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path edits, IOException e) throws IOException
IOExceptionprivate long replayRecoveredEdits(org.apache.hadoop.fs.Path edits, Map<byte[],Long> maxSeqIdInStores, CancelableProgressable reporter, org.apache.hadoop.fs.FileSystem fs) throws IOException
edits - File of recovered edits.maxSeqIdInStores - Maximum sequenceid found in each store. Edits in wal must be larger
          than this to be replayed for each store.minSeqId if nothing added from editlogs.IOExceptionvoid replayWALCompactionMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor compaction, boolean pickCompactionFiles, boolean removeFiles, long replaySeqId) throws IOException
IOExceptionvoid replayWALFlushMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush, long replaySeqId) throws IOException
IOExceptionHRegion.PrepareFlushResult replayWALFlushStartMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush) throws IOException
IOExceptionvoid replayWALFlushCommitMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush) throws IOException
IOExceptionprivate void replayFlushInStores(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush, HRegion.PrepareFlushResult prepareFlushResult, boolean dropMemstoreSnapshot) throws IOException
flush - prepareFlushResult - dropMemstoreSnapshot - IOExceptionprivate long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException
IOExceptionpublic MemStoreSize dropMemStoreContents() throws IOException
IOExceptionprivate MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException
IOExceptionprivate MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId) throws IOException
IOExceptionprivate void replayWALFlushAbortMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush)
private void replayWALFlushCannotFlushMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor flush, long replaySeqId)
HRegion.PrepareFlushResult getPrepareFlushResult()
void replayWALRegionEventMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor regionEvent) throws IOException
IOExceptionvoid replayWALBulkLoadEventMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException
IOExceptionprivate void dropPrepareFlushIfPossible()
public boolean refreshStoreFiles() throws IOException
RegionrefreshStoreFiles in interface RegionIOExceptionprotected boolean refreshStoreFiles(boolean force) throws IOException
IOExceptionprivate void logRegionFiles()
private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload) throws WrongRegionException
WrongRegionExceptionprotected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting)
s - Store to add edit too.cell - Cell to add.private static boolean isZeroLengthThenDelete(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.FileStatus stat, org.apache.hadoop.fs.Path p) throws IOException
p - File to check.IOExceptionprotected HStore instantiateHStore(ColumnFamilyDescriptor family, boolean warmup) throws IOException
IOExceptionpublic HStore getStore(byte[] column)
RegionUse with caution. Exposed for use of fixup utilities.
private HStore getStore(Cell cell)
public List<HStore> getStores()
RegionUse with caution. Exposed for use of fixup utilities.
public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException
getStoreFileList in interface RegionIllegalArgumentExceptionvoid checkRow(byte[] row, String op) throws IOException
IOExceptionpublic Region.RowLock getRowLock(byte[] row) throws IOException
row - Which row to lock.IOExceptionpublic Region.RowLock getRowLock(byte[] row, boolean readLock) throws IOException
Region
 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.
getRowLock 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)protected Region.RowLock getRowLockInternal(byte[] row, boolean readLock, Region.RowLock prevRowLock) throws IOException
IOExceptionprivate void releaseRowLocks(List<Region.RowLock> rowLocks)
public int getReadLockCount()
public ConcurrentHashMap<HashedBytes,HRegion.RowLockContext> getLockedRows()
private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[],String>> familyPaths)
familyPaths - List of (column family, hfilePath)public Map<byte[],List<org.apache.hadoop.fs.Path>> bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths, boolean assignSeqId, HRegion.BulkLoadListener bulkLoadListener) throws IOException
familyPaths - List of Pair<byte[] column family, String hfilePath>bulkLoadListener - Internal hooks enabling massaging/preparation of a
 file about to be bulk loadedassignSeqId - IOException - if failed unrecoverably.public Map<byte[],List<org.apache.hadoop.fs.Path>> bulkLoadHFiles(Collection<Pair<byte[],String>> familyPaths, boolean assignSeqId, HRegion.BulkLoadListener bulkLoadListener, boolean copyFile, List<String> clusterIds, boolean replicate) throws IOException
familyPaths - List of Pair<byte[] column family, String hfilePath>assignSeqId - bulkLoadListener - Internal hooks enabling massaging/preparation of a
 file about to be bulk loadedcopyFile - always copy hfiles if trueclusterIds - ids from clusters that had already handled the given bulkload event.IOException - if failed unrecoverably.public static HRegion newHRegion(org.apache.hadoop.fs.Path tableDir, WAL wal, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, RegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices)
HConstants.REGION_IMPL
 configuration property.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
          file (implying that the HRegion has been written-to before), then read it from the
          supplied path.fs - is the filesystem.conf - is global configuration settings.regionInfo - - RegionInfo that describes the region is new), then read them from the
          supplied path.htd - the table descriptorpublic static HRegion createHRegion(RegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, TableDescriptor hTableDescriptor, WAL wal, boolean initialize) throws IOException
info - Info for region to create.rootDir - Root directory for HBase instancewal - shared WALinitialize - - true to initialize the regionIOExceptionpublic static HRegion createHRegion(RegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, TableDescriptor hTableDescriptor, WAL wal, boolean initialize, RegionServerServices rsRpcServices) throws IOException
info - Info for region to create.rootDir - Root directory for HBase instancewal - shared WALinitialize - - true to initialize the regionrsRpcServices - An interface we can request flushes against.IOExceptionpublic static HRegion createHRegion(org.apache.hadoop.conf.Configuration conf, RegionInfo regionInfo, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, TableDescriptor tableDesc) throws IOException
IOExceptionpublic static HRegionFileSystem createRegionDir(org.apache.hadoop.conf.Configuration configuration, RegionInfo ri, org.apache.hadoop.fs.Path rootDir) throws IOException
IOExceptionpublic static HRegion createHRegion(RegionInfo info, org.apache.hadoop.fs.Path rootDir, org.apache.hadoop.conf.Configuration conf, TableDescriptor hTableDescriptor, WAL wal) throws IOException
IOExceptionpublic static HRegion openHRegion(RegionInfo info, TableDescriptor 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(RegionInfo info, TableDescriptor 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, RegionInfo info, TableDescriptor 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, RegionInfo info, TableDescriptor 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, RegionInfo info, TableDescriptor 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, RegionInfo info, TableDescriptor 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 openHRegionFromTableDir(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo info, TableDescriptor htd, WAL wal, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
conf - The Configuration object to use.fs - Filesystem to useinfo - 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 NavigableMap<byte[],Integer> getReplicationScope()
public 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, RegionInfo info, TableDescriptor htd) throws IOException
conf - The Configuration object to use.fs - Filesystem to useinfo - Info for region to be opened.htd - the table descriptorIOExceptionpublic static void warmupHRegion(RegionInfo info, TableDescriptor htd, WAL wal, org.apache.hadoop.conf.Configuration conf, RegionServerServices rsServices, CancelableProgressable reporter) throws IOException
IOException@Deprecated public static org.apache.hadoop.fs.Path getRegionDir(org.apache.hadoop.fs.Path tabledir, String name)
tabledir - qualified path for tablename - ENCODED region namepublic static boolean rowIsInRange(RegionInfo info, byte[] row)
info - RegionInfo that specifies the row rangerow - row to be checkedpublic static boolean rowIsInRange(RegionInfo info, byte[] row, int offset, short length)
public Result get(Get get) throws IOException
Regionget in interface Regionget - query parametersIOExceptionvoid prepareGet(Get get) throws IOException
IOExceptionpublic 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
IOExceptionvoid metricsUpdateForGet(List<Cell> results, long before)
public Result 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, 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.shaded.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")IOExceptionprivate void preProcess(RowProcessor<?,?> processor, WALEdit walEdit) throws IOException
IOExceptionprivate void doProcessRowWithTimeout(RowProcessor<?,?> processor, long now, HRegion region, List<Mutation> mutations, WALEdit walEdit, long timeout) throws IOException
IOExceptionpublic Result append(Append append) throws IOException
Regionappend in interface RegionIOExceptionpublic Result append(Append append, long nonceGroup, long nonce) throws IOException
IOExceptionpublic Result increment(Increment increment) throws IOException
Regionincrement in interface RegionIOExceptionpublic Result increment(Increment increment, long nonceGroup, long nonce) throws IOException
IOExceptionprivate MultiVersionConcurrencyControl.WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds, long now, long nonceGroup, long nonce) throws IOException
IOExceptionprivate MultiVersionConcurrencyControl.WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds, long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException
IOExceptionprivate static List<Cell> sort(List<Cell> cells, CellComparator comparator)
cells using comparatorvoid checkFamily(byte[] family) throws NoSuchColumnFamilyException
NoSuchColumnFamilyExceptionpublic long heapSize()
public boolean registerService(com.google.protobuf.Service instance)
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.
 
 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.
 
instance - 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.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall call) throws IOException
Service method using
 the registered protocol handlers.  Service implementations must be registered via the
 registerService(com.google.protobuf.Service)
 method before they are available.controller - 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 invocationregisterService(com.google.protobuf.Service)public Optional<byte[]> checkSplit()
public Optional<byte[]> checkSplit(boolean force)
public int getCompactPriority()
public RegionCoprocessorHost getCoprocessorHost()
public 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 RegionIOExceptionprivate void startBulkRegionOperation(boolean writeLockNeeded) throws IOException
NotServingRegionException - when the region is closing or closedRegionTooBusyException - if failed to get the lock in timeInterruptedIOException - if interrupted while waiting for a lockIOExceptionprivate void closeBulkRegionOperation()
private void recordMutationWithoutWal(Map<byte[],List<Cell>> familyMap)
private void lock(Lock lock) throws IOException
IOExceptionprivate void lock(Lock lock, int multiplier) throws IOException
IOExceptionprivate void sync(long txid, Durability durability) throws IOException
txid - should sync up to which transactionIOException - If anything goes wrong with DFSprivate boolean shouldSyncWAL()
public long getOpenSeqNum()
public Map<byte[],Long> getMaxStoreSeqId()
getMaxStoreSeqId in interface Regionpublic long getOldestSeqIdOfStore(byte[] familyName)
public 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 getReadPoint()
protected void disableInterrupts()
enableInterrupts().protected void enableInterrupts()
disableInterrupts(),
 make it eligible again. No-op if interrupts are already enabled.private void interruptRegionOperations()
startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation),
 or startBulkRegionOperation(boolean).void checkInterrupt() throws NotServingRegionException, InterruptedIOException
NotServingRegionException - if region is closingInterruptedIOException - if interrupted but region is not closingIOException throwOnInterrupt(Throwable t)
t - causepublic 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 CellComparator getCellComparator()
RegiongetCellComparator in interface Regionpublic long getMemStoreFlushSize()
void throwException(String title, String regionName)
public void requestCompaction(String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException
RegionrequestCompaction in interface RegionIOExceptionpublic void requestCompaction(byte[] family, String why, int priority, boolean major, CompactionLifeCycleTracker tracker) throws IOException
RegionrequestCompaction in interface RegionIOExceptionprivate void requestFlushIfNeeded() throws RegionTooBusyException
RegionTooBusyExceptionprivate void requestFlush()
private void requestFlush0(FlushLifeCycleTracker tracker)
public void requestFlush(FlushLifeCycleTracker tracker) throws IOException
RegionrequestFlush in interface RegionIOExceptionstatic void decorateRegionConfiguration(org.apache.hadoop.conf.Configuration conf)
conf - region configurationspublic void setReadRequestsCount(long readRequestsCount)
public void setWriteRequestsCount(long writeRequestsCount)
Copyright © 2007–2021 The Apache Software Foundation. All rights reserved.