@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public abstract class BaseRegionObserver extends Object implements RegionObserver
RegionObserver.MutationTypeCoprocessor.StatePRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION| Constructor and Description | 
|---|
BaseRegionObserver()  | 
| Modifier and Type | Method and Description | 
|---|---|
Result | 
postAppend(ObserverContext<RegionCoprocessorEnvironment> e,
          Append append,
          Result result)
Called after Append 
 | 
void | 
postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
               MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called after applying a batch of Mutations on a region. 
 | 
void | 
postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx,
                            MiniBatchOperationInProgress<Mutation> miniBatchOp,
                            boolean success)
Called after the completion of batch put/delete and will be called even if the batch operation
 fails 
 | 
boolean | 
postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
                 List<Pair<byte[],String>> stagingFamilyPaths,
                 boolean hasLoaded)
Called after bulkLoadHFile. 
 | 
boolean | 
postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
                 List<Pair<byte[],String>> stagingFamilyPaths,
                 Map<byte[],List<Path>> finalPaths,
                 boolean hasLoaded)
Called after bulkLoadHFile. 
 | 
boolean | 
postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e,
                  byte[] row,
                  byte[] family,
                  byte[] qualifier,
                  CompareFilter.CompareOp compareOp,
                  ByteArrayComparable comparator,
                  Delete delete,
                  boolean result)
Called after checkAndDelete 
 | 
boolean | 
postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e,
               byte[] row,
               byte[] family,
               byte[] qualifier,
               CompareFilter.CompareOp compareOp,
               ByteArrayComparable comparator,
               Put put,
               boolean result)
Called after checkAndPut 
 | 
void | 
postClose(ObserverContext<RegionCoprocessorEnvironment> e,
         boolean abortRequested)
Called after the region is reported as closed to the master. 
 | 
void | 
postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
                        Region.Operation op)
Called after releasing read lock in  
Region.closeRegionOperation(). | 
void | 
postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
                   byte[] family,
                   Path srcPath,
                   Path dstPath)
Called after moving bulk loaded hfile to region directory. 
 | 
void | 
postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
           Store store,
           StoreFile resultFile)
Called after compaction has completed and the new store file has been moved in to place. 
 | 
void | 
postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
           Store store,
           StoreFile resultFile,
           CompactionRequest request)
Called after compaction has completed and the new store file has been moved in to place. 
 | 
void | 
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
                    Store store,
                    <any> selected)
Called after the  
StoreFiles to compact have been selected from the available
 candidates. | 
void | 
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
                    Store store,
                    <any> selected,
                    CompactionRequest request)
Called after the  
StoreFiles to compact have been selected from the available
 candidates. | 
void | 
postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
Called after any split request is processed. 
 | 
void | 
postDelete(ObserverContext<RegionCoprocessorEnvironment> e,
          Delete delete,
          WALEdit edit,
          Durability durability)
Called after the client deletes a value. 
 | 
boolean | 
postExists(ObserverContext<RegionCoprocessorEnvironment> e,
          Get get,
          boolean exists)
Called after the client tests for existence using a Get. 
 | 
void | 
postFlush(ObserverContext<RegionCoprocessorEnvironment> e)
Called after the memstore is flushed to disk. 
 | 
void | 
postFlush(ObserverContext<RegionCoprocessorEnvironment> e,
         Store store,
         StoreFile resultFile)
Called after a Store's memstore is flushed to disk. 
 | 
void | 
postGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> e,
                       byte[] row,
                       byte[] family,
                       Result result)
Called after a client makes a GetClosestRowBefore request. 
 | 
void | 
postGetOp(ObserverContext<RegionCoprocessorEnvironment> e,
         Get get,
         List<Cell> results)
Called after the client performs a Get 
 | 
Result | 
postIncrement(ObserverContext<RegionCoprocessorEnvironment> e,
             Increment increment,
             Result result)
Called after increment 
 | 
long | 
postIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> e,
                        byte[] row,
                        byte[] family,
                        byte[] qualifier,
                        long amount,
                        boolean writeToWAL,
                        long result)
Called after incrementColumnValue 
 | 
DeleteTracker | 
postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx,
                            DeleteTracker delTracker)
Called after the ScanQueryMatcher creates ScanDeleteTracker. 
 | 
void | 
postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e)
Called after the log replay on the region is over. 
 | 
Cell | 
postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
                     RegionObserver.MutationType opType,
                     Mutation mutation,
                     Cell oldCell,
                     Cell newCell)
Called after a new cell has been created during an increment operation, but before
 it is committed to the WAL or memstore. 
 | 
void | 
postOpen(ObserverContext<RegionCoprocessorEnvironment> e)
Called after the region is reported as open to the master. 
 | 
void | 
postPut(ObserverContext<RegionCoprocessorEnvironment> e,
       Put put,
       WALEdit edit,
       Durability durability)
Called after the client stores a value. 
 | 
void | 
postRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called after the roll back of the split region is completed 
 | 
void | 
postScannerClose(ObserverContext<RegionCoprocessorEnvironment> e,
                InternalScanner s)
Called after the client closes a scanner. 
 | 
boolean | 
postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> e,
                    InternalScanner s,
                    byte[] currentRow,
                    int offset,
                    short length,
                    boolean hasMore)
This will be called by the scan flow when the current scanned row is being filtered out by the
 filter. 
 | 
boolean | 
postScannerNext(ObserverContext<RegionCoprocessorEnvironment> e,
               InternalScanner s,
               List<Result> results,
               int limit,
               boolean hasMore)
Called after the client asks for the next row on a scanner. 
 | 
RegionScanner | 
postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e,
               Scan scan,
               RegionScanner s)
Called after the client opens a new scanner. 
 | 
void | 
postSplit(ObserverContext<RegionCoprocessorEnvironment> e,
         Region l,
         Region r)
Called after the region is split. 
 | 
void | 
postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
                        Region.Operation op)
This will be called for region operations where read lock is acquired in
  
Region.startRegionOperation(). | 
StoreFile.Reader | 
postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
                       FileSystem fs,
                       Path p,
                       FSDataInputStreamWrapper in,
                       long size,
                       CacheConfig cacheConf,
                       Reference r,
                       StoreFile.Reader reader)
Called after the creation of Reader for a store file. 
 | 
void | 
postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
              HRegionInfo info,
              WALKey logKey,
              WALEdit logEdit)
Implementers should override this version of the method and leave the deprecated one as-is. 
 | 
void | 
postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
              HRegionInfo info,
              HLogKey logKey,
              WALEdit logEdit)
Called after a  
WALEdit
 replayed for this region. | 
Result | 
preAppend(ObserverContext<RegionCoprocessorEnvironment> e,
         Append append)
Called before Append. 
 | 
Result | 
preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
                     Append append)
Called before Append but after acquiring rowlock. 
 | 
void | 
preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
              MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called for every batch mutation operation happening at the server. 
 | 
void | 
preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
                List<Pair<byte[],String>> familyPaths)
Called before bulkLoadHFile. 
 | 
boolean | 
preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e,
                 byte[] row,
                 byte[] family,
                 byte[] qualifier,
                 CompareFilter.CompareOp compareOp,
                 ByteArrayComparable comparator,
                 Delete delete,
                 boolean result)
Called before checkAndDelete. 
 | 
boolean | 
preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
                             byte[] row,
                             byte[] family,
                             byte[] qualifier,
                             CompareFilter.CompareOp compareOp,
                             ByteArrayComparable comparator,
                             Delete delete,
                             boolean result)
Called before checkAndDelete but after acquiring rowock. 
 | 
boolean | 
preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e,
              byte[] row,
              byte[] family,
              byte[] qualifier,
              CompareFilter.CompareOp compareOp,
              ByteArrayComparable comparator,
              Put put,
              boolean result)
Called before checkAndPut. 
 | 
boolean | 
preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
                          byte[] row,
                          byte[] family,
                          byte[] qualifier,
                          CompareFilter.CompareOp compareOp,
                          ByteArrayComparable comparator,
                          Put put,
                          boolean result)
Called before checkAndPut but after acquiring rowlock. 
 | 
void | 
preClose(ObserverContext<RegionCoprocessorEnvironment> c,
        boolean abortRequested)
Called before the region is reported as closed to the master. 
 | 
void | 
preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
                  byte[] family,
                  List<Pair<Path,Path>> pairs)
Called before moving bulk loaded hfile to region directory. 
 | 
InternalScanner | 
preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
          Store store,
          InternalScanner scanner,
          ScanType scanType)
Called prior to writing the  
StoreFiles selected for compaction into a new
 StoreFile. | 
InternalScanner | 
preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
          Store store,
          InternalScanner scanner,
          ScanType scanType,
          CompactionRequest request)
Called prior to writing the  
StoreFiles selected for compaction into a new
 StoreFile. | 
InternalScanner | 
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
                     Store store,
                     List<? extends KeyValueScanner> scanners,
                     ScanType scanType,
                     long earliestPutTs,
                     InternalScanner s)
Called prior to writing the  
StoreFiles selected for compaction into a new
 StoreFile and prior to creating the scanner used to read the input files. | 
InternalScanner | 
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
                     Store store,
                     List<? extends KeyValueScanner> scanners,
                     ScanType scanType,
                     long earliestPutTs,
                     InternalScanner s,
                     CompactionRequest request)
Called prior to writing the  
StoreFiles selected for compaction into a new
 StoreFile and prior to creating the scanner used to read the input files. | 
void | 
preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
                   Store store,
                   List<StoreFile> candidates)
Called prior to selecting the  
StoreFiles to compact from the list of available
 candidates. | 
void | 
preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
                   Store store,
                   List<StoreFile> candidates,
                   CompactionRequest request)
Called prior to selecting the  
StoreFiles to compact from the list of
 available candidates. | 
void | 
preDelete(ObserverContext<RegionCoprocessorEnvironment> e,
         Delete delete,
         WALEdit edit,
         Durability durability)
Called before the client deletes a value. 
 | 
boolean | 
preExists(ObserverContext<RegionCoprocessorEnvironment> e,
         Get get,
         boolean exists)
Called before the client tests for existence using a Get. 
 | 
void | 
preFlush(ObserverContext<RegionCoprocessorEnvironment> e)
Called before the memstore is flushed to disk. 
 | 
InternalScanner | 
preFlush(ObserverContext<RegionCoprocessorEnvironment> e,
        Store store,
        InternalScanner scanner)
Called before a Store's memstore is flushed to disk. 
 | 
InternalScanner | 
preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
                   Store store,
                   KeyValueScanner memstoreScanner,
                   InternalScanner s)
Called before a memstore is flushed to disk and prior to creating the scanner to read from
 the memstore. 
 | 
void | 
preGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> e,
                      byte[] row,
                      byte[] family,
                      Result result)
Called before a client makes a GetClosestRowBefore request. 
 | 
void | 
preGetOp(ObserverContext<RegionCoprocessorEnvironment> e,
        Get get,
        List<Cell> results)
Called before the client performs a Get 
 | 
Result | 
preIncrement(ObserverContext<RegionCoprocessorEnvironment> e,
            Increment increment)
Called before Increment. 
 | 
Result | 
preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
                        Increment increment)
Called before Increment but after acquiring rowlock. 
 | 
long | 
preIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> e,
                       byte[] row,
                       byte[] family,
                       byte[] qualifier,
                       long amount,
                       boolean writeToWAL)
Called before incrementColumnValue 
 | 
void | 
preOpen(ObserverContext<RegionCoprocessorEnvironment> e)
Called before the region is reported as open to the master. 
 | 
void | 
prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> e,
                                   Mutation delete,
                                   Cell cell,
                                   byte[] byteNow,
                                   Get get)
Called before the server updates the timestamp for version delete with latest timestamp. 
 | 
void | 
prePut(ObserverContext<RegionCoprocessorEnvironment> e,
      Put put,
      WALEdit edit,
      Durability durability)
Called before the client stores a value. 
 | 
void | 
preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called before the roll back of the split region is completed 
 | 
void | 
preScannerClose(ObserverContext<RegionCoprocessorEnvironment> e,
               InternalScanner s)
Called before the client closes a scanner. 
 | 
boolean | 
preScannerNext(ObserverContext<RegionCoprocessorEnvironment> e,
              InternalScanner s,
              List<Result> results,
              int limit,
              boolean hasMore)
Called before the client asks for the next row on a scanner. 
 | 
RegionScanner | 
preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e,
              Scan scan,
              RegionScanner s)
Called before the client opens a new scanner. 
 | 
void | 
preSplit(ObserverContext<RegionCoprocessorEnvironment> e)
Called before the region is split. 
 | 
void | 
preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
        byte[] splitRow)
Called before the region is split. 
 | 
void | 
preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called after PONR step as part of split transaction
 Calling  
ObserverContext.bypass() has no
 effect in this hook. | 
void | 
preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
                  byte[] splitKey,
                  List<Mutation> metaEntries)
This will be called before PONR step as part of split transaction. 
 | 
StoreFile.Reader | 
preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
                      FileSystem fs,
                      Path p,
                      FSDataInputStreamWrapper in,
                      long size,
                      CacheConfig cacheConf,
                      Reference r,
                      StoreFile.Reader reader)
Called before creation of Reader for a store file. 
 | 
KeyValueScanner | 
preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
                   Store store,
                   Scan scan,
                   NavigableSet<byte[]> targetCols,
                   KeyValueScanner s)
Called before a store opens a new scanner. 
 | 
void | 
preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
             HRegionInfo info,
             WALKey logKey,
             WALEdit logEdit)
Implementers should override this version of the method and leave the deprecated one as-is. 
 | 
void | 
preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
             HRegionInfo info,
             HLogKey logKey,
             WALEdit logEdit)
Called before a  
WALEdit
 replayed for this region. | 
void | 
start(CoprocessorEnvironment e)  | 
void | 
stop(CoprocessorEnvironment e)  | 
public void start(CoprocessorEnvironment e) throws IOException
start in interface CoprocessorIOExceptionpublic void stop(CoprocessorEnvironment e) throws IOException
stop in interface CoprocessorIOExceptionpublic void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserverpreOpen in interface RegionObservere - the environment provided by the region serverIOException - if an error occurred on the coprocessorpublic void postOpen(ObserverContext<RegionCoprocessorEnvironment> e)
RegionObserverpostOpen in interface RegionObservere - the environment provided by the region serverpublic void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e)
RegionObserverpostLogReplay in interface RegionObservere - the environment provided by the region serverpublic void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException
RegionObserverpreClose in interface RegionObserverc - the environment provided by the region serverabortRequested - true if the region server is abortingIOExceptionpublic void postClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested)
RegionObserverpostClose in interface RegionObservere - the environment provided by the region serverabortRequested - true if the region server is abortingpublic InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException
RegionObserverStoreFile or null to perform the default processing.
 Calling ObserverContext.bypass() has no
 effect in this hook.preFlushScannerOpen in interface RegionObserverc - the environment provided by the region serverstore - the store being flushedmemstoreScanner - the scanner for the memstore that is flusheds - the base scanner, if not null, from previous RegionObserver in the chainnull if the default implementation
 is to be used.IOException - if an error occurred on the coprocessorpublic void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserverpreFlush in interface RegionObservere - the environment provided by the region serverIOException - if an error occurred on the coprocessorpublic void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserverpostFlush in interface RegionObservere - the environment provided by the region serverIOException - if an error occurred on the coprocessorpublic InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> e, Store store, InternalScanner scanner) throws IOException
RegionObserverpreFlush in interface RegionObservere - the environment provided by the region serverstore - the store where compaction is being requestedscanner - the scanner over existing data used in the store filenull
 unless the implementation is writing new store files on its own.IOException - if an error occurred on the coprocessorpublic void postFlush(ObserverContext<RegionCoprocessorEnvironment> e, Store store, StoreFile resultFile) throws IOException
RegionObserverpostFlush in interface RegionObservere - the environment provided by the region serverstore - the store being flushedresultFile - the new store file written out during compactionIOException - if an error occurred on the coprocessorpublic void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserverpreSplit in interface RegionObservere - the environment provided by the region server
 (e.getRegion() returns the parent region)IOException - if an error occurred on the coprocessorpublic void preSplit(ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow) throws IOException
RegionObserverpreSplit in interface RegionObserverc - the environment provided by the region server
 (e.getRegion() returns the parent region)IOException - if an error occurred on the coprocessorpublic void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) throws IOException
RegionObserverObserverContext.bypass() rollback the splitpreSplitBeforePONR in interface RegionObserverIOExceptionpublic void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserverObserverContext.bypass() has no
 effect in this hook.preSplitAfterPONR in interface RegionObserverIOExceptionpublic void preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserverpreRollBackSplit in interface RegionObserverIOExceptionpublic void postRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserverpostRollBackSplit in interface RegionObserverIOExceptionpublic void postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserverpostCompleteSplit in interface RegionObserverIOExceptionpublic void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r) throws IOException
RegionObserverpostSplit in interface RegionObservere - the environment provided by the region server
 (e.getRegion() returns the parent region)l - the left daughter regionr - the right daughter regionIOException - if an error occurred on the coprocessorpublic void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<StoreFile> candidates) throws IOException
RegionObserverStoreFiles to compact from the list of available
 candidates. To alter the files used for compaction, you may mutate the passed in list of
 candidates.preCompactSelection in interface RegionObserverc - the environment provided by the region serverstore - the store where compaction is being requestedcandidates - the store files currently available for compactionIOException - if an error occurred on the coprocessorpublic void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<StoreFile> candidates, CompactionRequest request) throws IOException
RegionObserverStoreFiles to compact from the list of
 available candidates. To alter the files used for compaction, you may mutate the passed in list
 of candidates.preCompactSelection in interface RegionObserverc - the environment provided by the region serverstore - the store where compaction is being requestedcandidates - the store files currently available for compactionrequest - custom compaction requestIOException - if an error occurred on the coprocessorpublic void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, <any> selected)
RegionObserverStoreFiles to compact have been selected from the available
 candidates.postCompactSelection in interface RegionObserverc - the environment provided by the region serverstore - the store being compactedselected - the store files selected to compactpublic void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, <any> selected, CompactionRequest request)
RegionObserverStoreFiles to compact have been selected from the available
 candidates.postCompactSelection in interface RegionObserverc - the environment provided by the region serverstore - the store being compactedselected - the store files selected to compactrequest - custom compaction requestpublic InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, InternalScanner scanner, ScanType scanType) throws IOException
RegionObserverStoreFiles selected for compaction into a new
 StoreFile. To override or modify the compaction process, implementing classes have two
 options:
 InternalScanner with a custom implementation that is returned
 from this method. The custom scanner can then inspect 
 KeyValues from the wrapped
 scanner, applying its own policy to what gets written.ObserverContext.bypass() and provide a
 custom implementation for writing of new StoreFiles. Note: any implementations
 bypassing core compaction using this approach must write out new store files themselves or the
 existing data will no longer be available after compaction.preCompact in interface RegionObservere - the environment provided by the region serverstore - the store being compactedscanner - the scanner over existing data used in the store file rewritingscanType - type of Scannull unless the
         implementation is writing new store files on its own.IOException - if an error occurred on the coprocessorpublic InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException
RegionObserverStoreFiles selected for compaction into a new
 StoreFile. To override or modify the compaction process, implementing classes have two
 options:
 InternalScanner with a custom implementation that is returned
 from this method. The custom scanner can then inspect 
 KeyValues from the wrapped
 scanner, applying its own policy to what gets written.ObserverContext.bypass() and provide a
 custom implementation for writing of new StoreFiles. Note: any implementations
 bypassing core compaction using this approach must write out new store files themselves or the
 existing data will no longer be available after compaction.preCompact in interface RegionObservere - the environment provided by the region serverstore - the store being compactedscanner - the scanner over existing data used in the store file rewritingscanType - type of Scanrequest - the requested compactionnull unless the
         implementation is writing new store files on its own.IOException - if an error occurred on the coprocessorpublic InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s) throws IOException
RegionObserverStoreFiles selected for compaction into a new
 StoreFile and prior to creating the scanner used to read the input files. To override
 or modify the compaction process, implementing classes can return a new scanner to provide the
 KeyValues to be stored into the new StoreFile or null to perform the default
 processing. Calling ObserverContext.bypass() has no
 effect in this hook.preCompactScannerOpen in interface RegionObserverc - the environment provided by the region serverstore - the store being compactedscanners - the list StoreFileScanners 
        to be read fromscanType - the ScanType indicating whether this is a major or minor compactionearliestPutTs - timestamp of the earliest put that was found in any of the involved store
          filess - the base scanner, if not null, from previous RegionObserver in the chainnull if the default implementation is to
         be used.IOException - if an error occurred on the coprocessorpublic InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException
RegionObserverStoreFiles selected for compaction into a new
 StoreFile and prior to creating the scanner used to read the input files. To override
 or modify the compaction process, implementing classes can return a new scanner to provide the
 KeyValues to be stored into the new StoreFile or null to perform the default
 processing. Calling ObserverContext.bypass() has no
 effect in this hook.preCompactScannerOpen in interface RegionObserverc - the environment provided by the region serverstore - the store being compactedscanners - the list StoreFileScanners 
        to be read fromscanType - the ScanType indicating whether this is a major or minor compactionearliestPutTs - timestamp of the earliest put that was found in any of the involved store
          filess - the base scanner, if not null, from previous RegionObserver in the chainrequest - the requested compactionnull if the default implementation is to
         be used.IOException - if an error occurred on the coprocessorpublic void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, StoreFile resultFile) throws IOException
RegionObserverpostCompact in interface RegionObservere - the environment provided by the region serverstore - the store being compactedresultFile - the new store file written out during compactionIOException - if an error occurred on the coprocessorpublic void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, StoreFile resultFile, CompactionRequest request) throws IOException
RegionObserverpostCompact in interface RegionObservere - the environment provided by the region serverstore - the store being compactedresultFile - the new store file written out during compactionrequest - the requested compactionIOException - if an error occurred on the coprocessorpublic void preGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, Result result) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preGetClosestRowBefore in interface RegionObservere - the environment provided by the region serverrow - the rowfamily - the familyresult - The result to return to the client if default processing
 is bypassed. Can be modified. Will not be used if default processing
 is not bypassed.IOException - if an error occurred on the coprocessorpublic void postGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, Result result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postGetClosestRowBefore in interface RegionObservere - the environment provided by the region serverrow - the rowfamily - the desired familyresult - the result to return to the client, modify as necessaryIOException - if an error occurred on the coprocessorpublic void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preGetOp in interface RegionObservere - the environment provided by the region serverget - the Get requestresults - The result to return to the client if default processing
 is bypassed. Can be modified. Will not be used if default processing
 is not bypassed.IOException - if an error occurred on the coprocessorpublic void postGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postGetOp in interface RegionObservere - the environment provided by the region serverget - the Get requestresults - the result to return to the client, modify as necessaryIOException - if an error occurred on the coprocessorpublic boolean preExists(ObserverContext<RegionCoprocessorEnvironment> e, Get get, boolean exists) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preExists in interface RegionObservere - the environment provided by the region serverget - the Get requestIOException - if an error occurred on the coprocessorpublic boolean postExists(ObserverContext<RegionCoprocessorEnvironment> e, Get get, boolean exists) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postExists in interface RegionObservere - the environment provided by the region serverget - the Get requestexists - the result returned by the region serverIOException - if an error occurred on the coprocessorpublic void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit, Durability durability) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
prePut in interface RegionObservere - the environment provided by the region serverput - The Put objectedit - The WALEdit object that will be written to the waldurability - Persistence guarantee for this PutIOException - if an error occurred on the coprocessorpublic void postPut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit, Durability durability) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postPut in interface RegionObservere - the environment provided by the region serverput - The Put objectedit - The WALEdit object for the waldurability - Persistence guarantee for this PutIOException - if an error occurred on the coprocessorpublic void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete, WALEdit edit, Durability durability) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preDelete in interface RegionObservere - the environment provided by the region serverdelete - The Delete objectedit - The WALEdit object for the waldurability - Persistence guarantee for this DeleteIOException - if an error occurred on the coprocessorpublic void prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> e, Mutation delete, Cell cell, byte[] byteNow, Get get) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
prePrepareTimeStampForDeleteVersion in interface RegionObservere - the environment provided by the region serverdelete - - the parent mutation associated with this delete cellcell - - The deleteColumn with latest version cellbyteNow - - timestamp bytesget - - the get formed using the current cell's row.
 Note that the get does not specify the family and qualifierIOExceptionpublic void postDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete, WALEdit edit, Durability durability) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postDelete in interface RegionObservere - the environment provided by the region serverdelete - The Delete objectedit - The WALEdit object for the waldurability - Persistence guarantee for this DeleteIOException - if an error occurred on the coprocessorpublic void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
RegionObserverMiniBatchOperationInProgress.setOperationStatus(int, OperationStatus)),
 RegionObserver can make Region to skip these Mutations.preBatchMutate in interface RegionObserverc - the environment provided by the region serverminiBatchOp - batch of Mutations getting applied to region.IOException - if an error occurred on the coprocessorpublic void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
RegionObserverpostBatchMutate in interface RegionObserverc - the environment provided by the region serverminiBatchOp - batch of Mutations applied to region.IOException - if an error occurred on the coprocessorpublic void postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx, MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException
RegionObserverpostBatchMutateIndispensably in interface RegionObserversuccess - true if batch operation is successful otherwise false.IOExceptionpublic boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndPut in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatorput - data to put if check succeedsIOException - if an error occurred on the coprocessorpublic boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserverNote: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndPutAfterRowLock in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatorput - data to put if check succeedsIOException - if an error occurred on the coprocessorpublic boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postCheckAndPut in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatorput - data to put if check succeedsresult - from the checkAndPutIOException - if an error occurred on the coprocessorpublic boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndDelete in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatordelete - delete to commit if check succeedsIOException - if an error occurred on the coprocessorpublic boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
RegionObserverNote: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndDeleteAfterRowLock in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatordelete - delete to commit if check succeedsIOException - if an error occurred on the coprocessorpublic boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postCheckAndDelete in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifiercompareOp - the comparison operationcomparator - the comparatordelete - delete to commit if check succeedsresult - from the CheckAndDeleteIOException - if an error occurred on the coprocessorpublic Result preAppend(ObserverContext<RegionCoprocessorEnvironment> e, Append append) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preAppend in interface RegionObservere - the environment provided by the region serverappend - Append objectIOException - if an error occurred on the coprocessorpublic Result preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e, Append append) throws IOException
RegionObserverNote: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preAppendAfterRowLock in interface RegionObservere - the environment provided by the region serverappend - Append objectIOException - if an error occurred on the coprocessorpublic Result postAppend(ObserverContext<RegionCoprocessorEnvironment> e, Append append, Result result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postAppend in interface RegionObservere - the environment provided by the region serverappend - Append objectresult - the result returned by incrementIOException - if an error occurred on the coprocessorpublic long preIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preIncrementColumnValue in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifieramount - long amount to incrementwriteToWAL - true if the change should be written to the WALIOException - if an error occurred on the coprocessorpublic long postIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL, long result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postIncrementColumnValue in interface RegionObservere - the environment provided by the region serverrow - row to checkfamily - column familyqualifier - column qualifieramount - long amount to incrementwriteToWAL - true if the change should be written to the WALresult - the result returned by incrementColumnValueIOException - if an error occurred on the coprocessorpublic Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> e, Increment increment) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preIncrement in interface RegionObservere - the environment provided by the region serverincrement - increment objectIOException - if an error occurred on the coprocessorpublic Result preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e, Increment increment) throws IOException
RegionObserverNote: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preIncrementAfterRowLock in interface RegionObservere - the environment provided by the region serverincrement - increment objectIOException - if an error occurred on the coprocessorpublic Result postIncrement(ObserverContext<RegionCoprocessorEnvironment> e, Increment increment, Result result) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postIncrement in interface RegionObservere - the environment provided by the region serverincrement - increment objectresult - the result returned by incrementIOException - if an error occurred on the coprocessorpublic RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan, RegionScanner s) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerOpen in interface RegionObservere - the environment provided by the region serverscan - the Scan specifications - if not null, the base scannerIOException - if an error occurred on the coprocessorpublic KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s) throws IOException
RegionObserver
 See RegionObserver.preFlushScannerOpen(ObserverContext, Store, KeyValueScanner, InternalScanner)
 and RegionObserver.preCompactScannerOpen(ObserverContext,
  Store, List, ScanType, long, InternalScanner)
 to override scanners created for flushes or compactions, resp.
 
 Call CoprocessorEnvironment#complete to skip any subsequent chained
 coprocessors.
 Calling ObserverContext.bypass() has no
 effect in this hook.
preStoreScannerOpen in interface RegionObserverc - the environment provided by the region serverstore - the store being scannedscan - the Scan specificationtargetCols - columns to be used in the scanners - the base scanner, if not null, from previous RegionObserver in the chainnull to use the default implementationIOException - if an error occurred on the coprocessorpublic RegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan, RegionScanner s) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerOpen in interface RegionObservere - the environment provided by the region serverscan - the Scan specifications - if not null, the base scannerIOException - if an error occurred on the coprocessorpublic boolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s, List<Result> results, int limit, boolean hasMore) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerNext in interface RegionObservere - the environment provided by the region servers - the scannerresults - The result to return to the client if default processing
 is bypassed. Can be modified. Will not be returned if default processing
 is not bypassed.limit - the maximum number of results to returnhasMore - the 'has more' indicationIOException - if an error occurred on the coprocessorpublic boolean postScannerNext(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s, List<Result> results, int limit, boolean hasMore) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerNext in interface RegionObservere - the environment provided by the region servers - the scannerresults - the result to return to the client, can be modifiedlimit - the maximum number of results to returnhasMore - the 'has more' indicationIOException - if an error occurred on the coprocessorpublic boolean postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s, byte[] currentRow, int offset, short length, boolean hasMore) throws IOException
RegionObserverboolean filterRowKey(byte [] buffer, int offset, int length) returning trueboolean filterRow() returning truevoid filterRow(List kvs)  removing all the kvs from the passed ListpostScannerFilterRow in interface RegionObservere - the environment provided by the region servers - the scannercurrentRow - The current rowkey which got filtered outoffset - offset to rowkeylength - length of rowkeyhasMore - the 'has more' indicationIOExceptionpublic void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s) throws IOException
RegionObserverCall CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerClose in interface RegionObservere - the environment provided by the region servers - the scannerIOException - if an error occurred on the coprocessorpublic void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s) throws IOException
RegionObserverCall CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerClose in interface RegionObservere - the environment provided by the region servers - the scannerIOException - if an error occurred on the coprocessorpublic void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env, HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
preWALRestore in interface RegionObserverIOExceptionpublic void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
RegionObserverWALEdit
 replayed for this region.
 This method is left in place to maintain binary compatibility with older
 RegionObservers. If an implementation directly overrides
 RegionObserver.preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit) then this version
 won't be called at all, barring problems with the Security Manager. To work correctly
 in the presence of a strict Security Manager, or in the case of an implementation that
 relies on a parent class to implement preWALRestore, you should implement this method
 as a call to the non-deprecated version.
 Users of this method will see all edits that can be treated as HLogKey. If there are
 edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
 on this method. If a coprocessor gets skipped because of this mechanism, a log message
 at ERROR will be generated per coprocessor on the logger for CoprocessorHost once per
 classloader.preWALRestore in interface RegionObserverIOExceptionpublic void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env, HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
postWALRestore in interface RegionObserverIOExceptionpublic void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
RegionObserverWALEdit
 replayed for this region.
 This method is left in place to maintain binary compatibility with older
 RegionObservers. If an implementation directly overrides
 RegionObserver.postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit) then this version
 won't be called at all, barring problems with the Security Manager. To work correctly
 in the presence of a strict Security Manager, or in the case of an implementation that
 relies on a parent class to implement preWALRestore, you should implement this method
 as a call to the non-deprecated version.
 Users of this method will see all edits that can be treated as HLogKey. If there are
 edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
 on this method. If a coprocessor gets skipped because of this mechanism, a log message
 at ERROR will be generated per coprocessor on the logger for CoprocessorHost once per
 classloader.postWALRestore in interface RegionObserverIOExceptionpublic void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
RegionObserverpreBulkLoadHFile in interface RegionObserverfamilyPaths - pairs of { CF, HFile path } submitted for bulk load. Adding
 or removing from this list will add or remove HFiles to be bulk loaded.IOExceptionpublic void preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family, List<Pair<Path,Path>> pairs) throws IOException
RegionObserverpreCommitStoreFile in interface RegionObserverfamily - column familypairs - List of pairs of { HFile location in staging dir, HFile path in region dir }
 Each pair are for the same hfile.IOExceptionpublic void postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family, Path srcPath, Path dstPath) throws IOException
RegionObserverpostCommitStoreFile in interface RegionObserverfamily - column familysrcPath - Path to file before the movedstPath - Path to file after the moveIOExceptionpublic boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> stagingFamilyPaths, Map<byte[],List<Path>> finalPaths, boolean hasLoaded) throws IOException
RegionObserverpostBulkLoadHFile in interface RegionObserverstagingFamilyPaths - pairs of { CF, HFile path } submitted for bulk loadfinalPaths - Map of CF to List of file paths for the final loaded fileshasLoaded - whether the bulkLoad was successfulIOExceptionpublic boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> stagingFamilyPaths, boolean hasLoaded) throws IOException
RegionObserverpostBulkLoadHFile in interface RegionObserverstagingFamilyPaths - pairs of { CF, HFile path } submitted for bulk loadhasLoaded - whether the bulkLoad was successfulIOExceptionpublic StoreFile.Reader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFile.Reader reader) throws IOException
RegionObserverObserverContext.bypass() has no
 effect in this hook.preStoreFileReaderOpen in interface RegionObserverctx - the environment provided by the region serverfs - fileystem to read fromp - path to the filein - FSDataInputStreamWrappersize - Full size of the filer - original reference file. This will be not null only when reading a split file.reader - the base reader, if not null, from previous RegionObserver in the chainIOExceptionpublic StoreFile.Reader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFile.Reader reader) throws IOException
RegionObserverpostStoreFileReaderOpen in interface RegionObserverctx - the environment provided by the region serverfs - fileystem to read fromp - path to the filein - FSDataInputStreamWrappersize - Full size of the filer - original reference file. This will be not null only when reading a split file.reader - the base reader instanceIOExceptionpublic Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException
RegionObserverObserverContext.bypass() has no
 effect in this hook.postMutationBeforeWAL in interface RegionObserverctx - the environment provided by the region serveropType - the operation typemutation - the current mutationoldCell - old cell containing previous valuenewCell - the new cell containing the computed valueIOExceptionpublic void postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation op) throws IOException
RegionObserverRegion.startRegionOperation().postStartRegionOperation in interface RegionObserverop - The operation is about to be taken on the regionIOExceptionpublic void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation op) throws IOException
RegionObserverRegion.closeRegionOperation().postCloseRegionOperation in interface RegionObserverIOExceptionpublic DeleteTracker postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException
RegionObserverpostInstantiateDeleteTracker in interface RegionObserverctx - the environment provided by the region serverdelTracker - the deleteTracker that is created by the QueryMatcherIOException