@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public abstract class BaseRegionObserver extends Object implements RegionObserver
RegionObserver.MutationType
Coprocessor.State
PRIORITY_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>> familyPaths,
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 |
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,
com.google.common.collect.ImmutableList<StoreFile> selected)
Called after the
StoreFile s to compact have been selected from the available
candidates. |
void |
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
com.google.common.collect.ImmutableList<StoreFile> selected,
CompactionRequest request)
Called after the
StoreFile s 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,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.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.
|
InternalScanner |
preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store,
InternalScanner scanner,
ScanType scanType)
Called prior to writing the
StoreFile s 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
StoreFile s 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
StoreFile s 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
StoreFile s 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
StoreFile s 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,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.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 Coprocessor
IOException
public void stop(CoprocessorEnvironment e) throws IOException
stop
in interface Coprocessor
IOException
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserver
preOpen
in interface RegionObserver
e
- the environment provided by the region serverIOException
- if an error occurred on the coprocessorpublic void postOpen(ObserverContext<RegionCoprocessorEnvironment> e)
RegionObserver
postOpen
in interface RegionObserver
e
- the environment provided by the region serverpublic void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e)
RegionObserver
postLogReplay
in interface RegionObserver
e
- the environment provided by the region serverpublic void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException
RegionObserver
preClose
in interface RegionObserver
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingIOException
public void postClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested)
RegionObserver
postClose
in interface RegionObserver
e
- 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
RegionObserver
StoreFile
or null to perform the default processing.
Calling ObserverContext.bypass()
has no
effect in this hook.preFlushScannerOpen
in interface RegionObserver
c
- 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
RegionObserver
preFlush
in interface RegionObserver
e
- the environment provided by the region serverIOException
- if an error occurred on the coprocessorpublic void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException
RegionObserver
postFlush
in interface RegionObserver
e
- 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
RegionObserver
preFlush
in interface RegionObserver
e
- 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
RegionObserver
postFlush
in interface RegionObserver
e
- 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
RegionObserver
preSplit
in interface RegionObserver
e
- 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
RegionObserver
preSplit
in interface RegionObserver
c
- 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
RegionObserver
ObserverContext.bypass()
rollback the splitpreSplitBeforePONR
in interface RegionObserver
IOException
public void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserver
ObserverContext.bypass()
has no
effect in this hook.preSplitAfterPONR
in interface RegionObserver
IOException
public void preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserver
preRollBackSplit
in interface RegionObserver
IOException
public void postRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserver
postRollBackSplit
in interface RegionObserver
IOException
public void postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
RegionObserver
postCompleteSplit
in interface RegionObserver
IOException
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r) throws IOException
RegionObserver
postSplit
in interface RegionObserver
e
- 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
RegionObserver
StoreFile
s 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 RegionObserver
c
- 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
RegionObserver
StoreFiles
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 RegionObserver
c
- 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, com.google.common.collect.ImmutableList<StoreFile> selected)
RegionObserver
StoreFile
s to compact have been selected from the available
candidates.postCompactSelection
in interface RegionObserver
c
- the environment provided by the region serverstore
- the store being compactedselected
- the store files selected to compactpublic void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, com.google.common.collect.ImmutableList<StoreFile> selected, CompactionRequest request)
RegionObserver
StoreFile
s to compact have been selected from the available
candidates.postCompactSelection
in interface RegionObserver
c
- 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
RegionObserver
StoreFile
s 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
KeyValue
s from the wrapped
scanner, applying its own policy to what gets written.ObserverContext.bypass()
and provide a
custom implementation for writing of new StoreFile
s. 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 RegionObserver
e
- 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
RegionObserver
StoreFile
s 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
KeyValue
s from the wrapped
scanner, applying its own policy to what gets written.ObserverContext.bypass()
and provide a
custom implementation for writing of new StoreFile
s. 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 RegionObserver
e
- 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
RegionObserver
StoreFile
s 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 RegionObserver
c
- the environment provided by the region serverstore
- the store being compactedscanners
- the list StoreFileScanner
s
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
RegionObserver
StoreFile
s 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 RegionObserver
c
- the environment provided by the region serverstore
- the store being compactedscanners
- the list StoreFileScanner
s
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
RegionObserver
postCompact
in interface RegionObserver
e
- 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
RegionObserver
postCompact
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preGetClosestRowBefore
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postGetClosestRowBefore
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preGetOp
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postGetOp
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preExists
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postExists
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
prePut
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postPut
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preDelete
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
prePrepareTimeStampForDeleteVersion
in interface RegionObserver
e
- 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 qualifierIOException
public void postDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete, WALEdit edit, Durability durability) throws IOException
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postDelete
in interface RegionObserver
e
- 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
RegionObserver
MiniBatchOperationInProgress.setOperationStatus(int, OperationStatus)
),
RegionObserver
can make Region to skip these Mutations.preBatchMutate
in interface RegionObserver
c
- 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
RegionObserver
postBatchMutate
in interface RegionObserver
c
- 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
RegionObserver
postBatchMutateIndispensably
in interface RegionObserver
success
- true if batch operation is successful otherwise false.IOException
public boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndPut
in interface RegionObserver
e
- 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
RegionObserver
Note: 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 RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postCheckAndPut
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preCheckAndDelete
in interface RegionObserver
e
- 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
RegionObserver
Note: 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 RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postCheckAndDelete
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preAppend
in interface RegionObserver
e
- 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
RegionObserver
Note: 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 RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postAppend
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preIncrementColumnValue
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postIncrementColumnValue
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preIncrement
in interface RegionObserver
e
- 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
RegionObserver
Note: 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 RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postIncrement
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerOpen
in interface RegionObserver
e
- 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 RegionObserver
c
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerOpen
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerNext
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerNext
in interface RegionObserver
e
- 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
RegionObserver
boolean 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 RegionObserver
e
- 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' indicationIOException
public void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
preScannerClose
in interface RegionObserver
e
- 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
RegionObserver
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
postScannerClose
in interface RegionObserver
e
- 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 RegionObserver
IOException
public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
RegionObserver
WALEdit
replayed for this region.
This method is left in place to maintain binary compatibility with older
RegionObserver
s. 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 RegionObserver
IOException
public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env, HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
postWALRestore
in interface RegionObserver
IOException
public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
RegionObserver
WALEdit
replayed for this region.
This method is left in place to maintain binary compatibility with older
RegionObserver
s. 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 RegionObserver
IOException
public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
RegionObserver
preBulkLoadHFile
in interface RegionObserver
familyPaths
- pairs of { CF, HFile path } submitted for bulk load. Adding
or removing from this list will add or remove HFiles to be bulk loaded.IOException
public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths, boolean hasLoaded) throws IOException
RegionObserver
postBulkLoadHFile
in interface RegionObserver
familyPaths
- pairs of { CF, HFile path } submitted for bulk loadhasLoaded
- whether the bulkLoad was successfulIOException
public StoreFile.Reader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFile.Reader reader) throws IOException
RegionObserver
ObserverContext.bypass()
has no
effect in this hook.preStoreFileReaderOpen
in interface RegionObserver
ctx
- the environment provided by the region serverfs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- 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 chainIOException
public StoreFile.Reader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFile.Reader reader) throws IOException
RegionObserver
postStoreFileReaderOpen
in interface RegionObserver
ctx
- the environment provided by the region serverfs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- Full size of the filer
- original reference file. This will be not null only when reading a split file.reader
- the base reader instanceIOException
public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException
RegionObserver
ObserverContext.bypass()
has no
effect in this hook.postMutationBeforeWAL
in interface RegionObserver
ctx
- the environment provided by the region serveropType
- the operation typemutation
- the current mutationoldCell
- old cell containing previous valuenewCell
- the new cell containing the computed valueIOException
public void postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation op) throws IOException
RegionObserver
Region.startRegionOperation()
.postStartRegionOperation
in interface RegionObserver
op
- The operation is about to be taken on the regionIOException
public void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation op) throws IOException
RegionObserver
Region.closeRegionOperation()
.postCloseRegionOperation
in interface RegionObserver
IOException
public DeleteTracker postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException
RegionObserver
postInstantiateDeleteTracker
in interface RegionObserver
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcherIOException
Copyright © 2007-2016 The Apache Software Foundation. All Rights Reserved.