@InterfaceAudience.Private public class RegionCoprocessorHost extends CoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
Region
.Modifier and Type | Class and Description |
---|---|
(package private) class |
RegionCoprocessorHost.BulkLoadObserverOperation |
private static class |
RegionCoprocessorHost.RegionEnvironment
Encapsulation of the environment of each coprocessor
|
private static class |
RegionCoprocessorHost.RegionEnvironmentForCoreCoprocessors
Special version of RegionEnvironment that exposes RegionServerServices for Core Coprocessors
only.
|
(package private) class |
RegionCoprocessorHost.RegionObserverOperationWithoutResult |
(package private) static class |
RegionCoprocessorHost.TableCoprocessorAttribute |
CoprocessorHost.ObserverGetter<C,O>, CoprocessorHost.ObserverOperationWithoutResult<O>, CoprocessorHost.ObserverOperationWithResult<O,R>
Modifier and Type | Field and Description |
---|---|
private CoprocessorHost.ObserverGetter<RegionCoprocessor,EndpointObserver> |
endpointObserverGetter |
private boolean |
hasCustomPostScannerFilterRow |
private static org.slf4j.Logger |
LOG |
(package private) HRegion |
region
The region
|
private CoprocessorHost.ObserverGetter<RegionCoprocessor,RegionObserver> |
regionObserverGetter |
(package private) RegionServerServices |
rsServices
The region server services
|
private static org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap<String,ConcurrentMap<String,Object>> |
SHARED_DATA_MAP |
ABORT_ON_ERROR_KEY, abortable, conf, coprocEnvironments, COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_ABORT_ON_ERROR, DEFAULT_COPROCESSORS_ENABLED, DEFAULT_SKIP_LOAD_DUPLICATE_TABLE_COPROCESSOR, DEFAULT_USER_COPROCESSORS_ENABLED, loadSequence, MASTER_COPROCESSOR_CONF_KEY, pathPrefix, REGION_COPROCESSOR_CONF_KEY, REGIONSERVER_COPROCESSOR_CONF_KEY, SKIP_LOAD_DUPLICATE_TABLE_COPROCESSOR, USER_COPROCESSORS_ENABLED_CONF_KEY, USER_REGION_COPROCESSOR_CONF_KEY, WAL_COPROCESSOR_CONF_KEY
Constructor and Description |
---|
RegionCoprocessorHost(HRegion region,
RegionServerServices rsServices,
org.apache.hadoop.conf.Configuration conf)
Constructor
|
Modifier and Type | Method and Description |
---|---|
RegionCoprocessor |
checkAndGetInstance(Class<?> implClass)
Called when a new Coprocessor class needs to be loaded.
|
RegionCoprocessorHost.RegionEnvironment |
createEnvironment(RegionCoprocessor instance,
int priority,
int seq,
org.apache.hadoop.conf.Configuration conf)
Called when a new Coprocessor class is loaded
|
(package private) static List<RegionCoprocessorHost.TableCoprocessorAttribute> |
getTableCoprocessorAttrsFromSchema(org.apache.hadoop.conf.Configuration conf,
TableDescriptor htd) |
boolean |
hasCustomPostScannerFilterRow() |
(package private) void |
loadTableCoprocessors(org.apache.hadoop.conf.Configuration conf) |
Result |
postAppend(Append append,
Result result,
WALEdit edit) |
List<Pair<Cell,Cell>> |
postAppendBeforeWAL(Mutation mutation,
List<Pair<Cell,Cell>> cellPairs) |
void |
postBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) |
void |
postBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp,
boolean success) |
void |
postBulkLoadHFile(List<Pair<byte[],String>> familyPaths,
Map<byte[],List<org.apache.hadoop.fs.Path>> map) |
CheckAndMutateResult |
postCheckAndMutate(CheckAndMutate checkAndMutate,
CheckAndMutateResult result) |
void |
postClose(boolean abortRequested)
Invoked after a region is closed
|
void |
postCloseRegionOperation(Region.Operation op) |
void |
postCommitStoreFile(byte[] family,
org.apache.hadoop.fs.Path srcPath,
org.apache.hadoop.fs.Path dstPath) |
void |
postCompact(HStore store,
HStoreFile resultFile,
CompactionLifeCycleTracker tracker,
CompactionRequest request,
User user)
Called after the store compaction has completed.
|
void |
postCompactSelection(HStore store,
List<HStoreFile> selected,
CompactionLifeCycleTracker tracker,
CompactionRequest request,
User user)
Called after the
HStoreFile s to be compacted have been selected from the available
candidates. |
void |
postDelete(Delete delete,
WALEdit edit) |
void |
postEndpointInvocation(com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request,
com.google.protobuf.Message.Builder responseBuilder) |
boolean |
postExists(Get get,
boolean result) |
void |
postFlush(FlushLifeCycleTracker tracker)
Invoked after a memstore flush n
|
void |
postFlush(HStore store,
HStoreFile storeFile,
FlushLifeCycleTracker tracker)
Invoked after a memstore flush n
|
void |
postGet(Get get,
List<Cell> results) |
Result |
postIncrement(Increment increment,
Result result,
WALEdit edit) |
List<Pair<Cell,Cell>> |
postIncrementBeforeWAL(Mutation mutation,
List<Pair<Cell,Cell>> cellPairs) |
DeleteTracker |
postInstantiateDeleteTracker(DeleteTracker result)
Deprecated.
Since 2.0 with out any replacement and will be removed in 3.0
|
void |
postMemStoreCompaction(HStore store)
Invoked after in memory compaction.
|
void |
postOpen()
Invoked after a region open
|
void |
postPut(Put put,
WALEdit edit) |
void |
postReplayWALs(RegionInfo info,
org.apache.hadoop.fs.Path edits) |
void |
postScannerClose(InternalScanner s) |
boolean |
postScannerFilterRow(InternalScanner s,
Cell curRowCell)
This will be called by the scan flow when the current scanned row is being filtered out by the
filter.
|
boolean |
postScannerNext(InternalScanner s,
List<Result> results,
int limit,
boolean hasMore) |
RegionScanner |
postScannerOpen(Scan scan,
RegionScanner s) |
void |
postStartRegionOperation(Region.Operation op) |
StoreFileReader |
postStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFileReader reader) |
void |
postWALRestore(RegionInfo info,
WALKey logKey,
WALEdit logEdit)
Deprecated.
Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced with
something that doesn't expose IntefaceAudience.Private classes.
|
Result |
preAppend(Append append,
WALEdit edit)
Supports Coprocessor 'bypass'.
|
Result |
preAppendAfterRowLock(Append append)
Supports Coprocessor 'bypass'.
|
void |
preBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) |
void |
preBulkLoadHFile(List<Pair<byte[],String>> familyPaths) |
CheckAndMutateResult |
preCheckAndMutate(CheckAndMutate checkAndMutate)
Supports Coprocessor 'bypass'.
|
CheckAndMutateResult |
preCheckAndMutateAfterRowLock(CheckAndMutate checkAndMutate)
Supports Coprocessor 'bypass'.
|
void |
preCleanupBulkLoad(User user) |
void |
preClose(boolean abortRequested)
Invoked before a region is closed
|
boolean |
preCommitStoreFile(byte[] family,
List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs) |
InternalScanner |
preCompact(HStore store,
InternalScanner scanner,
ScanType scanType,
CompactionLifeCycleTracker tracker,
CompactionRequest request,
User user)
Called prior to rewriting the store files selected for compaction
|
ScanInfo |
preCompactScannerOpen(HStore store,
ScanType scanType,
CompactionLifeCycleTracker tracker,
CompactionRequest request,
User user)
Called prior to opening store scanner for compaction.
|
boolean |
preCompactSelection(HStore store,
List<HStoreFile> candidates,
CompactionLifeCycleTracker tracker,
User user)
Called prior to selecting the
HStoreFile s for compaction from the list of currently
available candidates. |
boolean |
preDelete(Delete delete,
WALEdit edit)
Supports Coprocessor 'bypass'.
|
com.google.protobuf.Message |
preEndpointInvocation(com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request) |
Boolean |
preExists(Get get)
Supports Coprocessor 'bypass'.
|
void |
preFlush(FlushLifeCycleTracker tracker)
Invoked before a memstore flush n
|
InternalScanner |
preFlush(HStore store,
InternalScanner scanner,
FlushLifeCycleTracker tracker)
Invoked before a memstore flush
|
ScanInfo |
preFlushScannerOpen(HStore store,
FlushLifeCycleTracker tracker)
Invoked before create StoreScanner for flush.
|
boolean |
preGet(Get get,
List<Cell> results)
Supports Coprocessor 'bypass'.
|
Result |
preIncrement(Increment increment,
WALEdit edit)
Supports Coprocessor 'bypass'.
|
Result |
preIncrementAfterRowLock(Increment increment)
Supports Coprocessor 'bypass'.
|
void |
preMemStoreCompaction(HStore store)
Invoked before in memory compaction.
|
InternalScanner |
preMemStoreCompactionCompact(HStore store,
InternalScanner scanner)
Invoked before compacting memstore.
|
ScanInfo |
preMemStoreCompactionCompactScannerOpen(HStore store)
Invoked before create StoreScanner for in memory compaction.
|
void |
preOpen()
Invoked before a region open.
|
void |
prePrepareBulkLoad(User user) |
boolean |
prePrepareTimeStampForDeleteVersion(Mutation mutation,
Cell kv,
byte[] byteNow,
Get get)
Deprecated.
In hbase-2.0.0. Will be removed in hbase-3.0.0. Added explicitly for a single
Coprocessor for its needs only. Will be removed.
|
boolean |
prePut(Put put,
WALEdit edit)
Supports Coprocessor 'bypass'.
|
void |
preReplayWALs(RegionInfo info,
org.apache.hadoop.fs.Path edits) |
boolean |
preScannerClose(InternalScanner s)
Supports Coprocessor 'bypass'.
|
Boolean |
preScannerNext(InternalScanner s,
List<Result> results,
int limit) |
void |
preScannerOpen(Scan scan) |
StoreFileReader |
preStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r) |
ScanInfo |
preStoreScannerOpen(HStore store,
Scan scan)
Called before open store scanner for user scan.
|
void |
preWALAppend(WALKey key,
WALEdit edit) |
boolean |
preWALRestore(RegionInfo info,
WALKey logKey,
WALEdit logEdit)
Deprecated.
Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced with
something that doesn't expose IntefaceAudience.Private classes.
|
static void |
testTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf,
TableDescriptor htd)
Sanity check the table coprocessor attributes of the supplied schema.
|
abortServer, abortServer, checkAndLoadInstance, execOperation, execOperationWithResult, execShutdown, findCoprocessor, findCoprocessor, findCoprocessorEnvironment, findCoprocessors, getCoprocessors, getLoadedCoprocessors, handleCoprocessorThrowable, load, load, load, loadSystemCoprocessors, shutdown
private static final org.slf4j.Logger LOG
private static final org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap<String,ConcurrentMap<String,Object>> SHARED_DATA_MAP
private final boolean hasCustomPostScannerFilterRow
RegionServerServices rsServices
private CoprocessorHost.ObserverGetter<RegionCoprocessor,RegionObserver> regionObserverGetter
private CoprocessorHost.ObserverGetter<RegionCoprocessor,EndpointObserver> endpointObserverGetter
public RegionCoprocessorHost(HRegion region, RegionServerServices rsServices, org.apache.hadoop.conf.Configuration conf)
region
- the regionrsServices
- interface to available region server functionalityconf
- the configurationpublic boolean hasCustomPostScannerFilterRow()
static List<RegionCoprocessorHost.TableCoprocessorAttribute> getTableCoprocessorAttrsFromSchema(org.apache.hadoop.conf.Configuration conf, TableDescriptor htd)
public static void testTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf, TableDescriptor htd) throws IOException
IOException
void loadTableCoprocessors(org.apache.hadoop.conf.Configuration conf)
public RegionCoprocessorHost.RegionEnvironment createEnvironment(RegionCoprocessor instance, int priority, int seq, org.apache.hadoop.conf.Configuration conf)
CoprocessorHost
createEnvironment
in class CoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
public RegionCoprocessor checkAndGetInstance(Class<?> implClass) throws InstantiationException, IllegalAccessException
CoprocessorHost
checkAndGetInstance
in class CoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
InstantiationException
IllegalAccessException
public void preOpen() throws IOException
IOException
- Signals that an I/O exception has occurred.public void postOpen()
public void preClose(boolean abortRequested) throws IOException
abortRequested
- true if the server is abortingIOException
public void postClose(boolean abortRequested)
abortRequested
- true if the server is abortingpublic boolean preCompactSelection(HStore store, List<HStoreFile> candidates, CompactionLifeCycleTracker tracker, User user) throws IOException
HStoreFile
s for compaction from the list of currently
available candidates.
Supports Coprocessor 'bypass' -- 'bypass' is how this method indicates that it changed the
passed in candidates
.
store
- The store where compaction is being requestedcandidates
- The currently available store filestracker
- used to track the life cycle of a compactionuser
- the user nIOException
public void postCompactSelection(HStore store, List<HStoreFile> selected, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException
HStoreFile
s to be compacted have been selected from the available
candidates.store
- The store where compaction is being requestedselected
- The store files selected to compacttracker
- used to track the life cycle of a compactionrequest
- the compaction requestuser
- the userIOException
public ScanInfo preCompactScannerOpen(HStore store, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException
IOException
public InternalScanner preCompact(HStore store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException
store
- the store being compactedscanner
- the scanner used to read store data during compactionscanType
- type of Scantracker
- used to track the life cycle of a compactionrequest
- the compaction requestuser
- the userIOException
public void postCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException
store
- the store being compactedresultFile
- the new store file written during compactiontracker
- used to track the life cycle of a compactionrequest
- the compaction requestuser
- the user nIOException
public ScanInfo preFlushScannerOpen(HStore store, FlushLifeCycleTracker tracker) throws IOException
IOException
public InternalScanner preFlush(HStore store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException
IOException
public void preFlush(FlushLifeCycleTracker tracker) throws IOException
IOException
public void postFlush(FlushLifeCycleTracker tracker) throws IOException
IOException
public void preMemStoreCompaction(HStore store) throws IOException
IOException
public ScanInfo preMemStoreCompactionCompactScannerOpen(HStore store) throws IOException
IOException
public InternalScanner preMemStoreCompactionCompact(HStore store, InternalScanner scanner) throws IOException
IOException
public void postMemStoreCompaction(HStore store) throws IOException
IOException
public void postFlush(HStore store, HStoreFile storeFile, FlushLifeCycleTracker tracker) throws IOException
IOException
public boolean preGet(Get get, List<Cell> results) throws IOException
get
- the Get requestresults
- What to return if return is true/'bypass'.IOException
- Exceptionpublic void postGet(Get get, List<Cell> results) throws IOException
get
- the Get requestresults
- the result setIOException
- Exceptionpublic Boolean preExists(Get get) throws IOException
get
- the Get requestIOException
- Exceptionpublic boolean postExists(Get get, boolean result) throws IOException
get
- the Get requestresult
- the result returned by the region serverIOException
- Exceptionpublic boolean prePut(Put put, WALEdit edit) throws IOException
put
- The Put objectedit
- The WALEdit object.IOException
- Exception@Deprecated public boolean prePrepareTimeStampForDeleteVersion(Mutation mutation, Cell kv, byte[] byteNow, Get get) throws IOException
mutation
- - the current mutationkv
- - the current cellbyteNow
- - current timestamp in bytesget
- - the get that could be used Note that the get only does not specify the family
and qualifier that should be usedIOException
public void postPut(Put put, WALEdit edit) throws IOException
put
- The Put objectedit
- The WALEdit object.IOException
- Exceptionpublic boolean preDelete(Delete delete, WALEdit edit) throws IOException
delete
- The Delete objectedit
- The WALEdit object.IOException
- Exceptionpublic void postDelete(Delete delete, WALEdit edit) throws IOException
delete
- The Delete objectedit
- The WALEdit object.IOException
- Exceptionpublic void preBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
IOException
public void postBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
IOException
public void postBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException
IOException
public CheckAndMutateResult preCheckAndMutate(CheckAndMutate checkAndMutate) throws IOException
checkAndMutate
- the CheckAndMutate objectIOException
- if an error occurred on the coprocessorpublic CheckAndMutateResult preCheckAndMutateAfterRowLock(CheckAndMutate checkAndMutate) throws IOException
checkAndMutate
- the CheckAndMutate objectIOException
- if an error occurred on the coprocessorpublic CheckAndMutateResult postCheckAndMutate(CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException
checkAndMutate
- the CheckAndMutate objectresult
- the result returned by the checkAndMutateIOException
- if an error occurred on the coprocessorpublic Result preAppend(Append append, WALEdit edit) throws IOException
append
- append objectedit
- The WALEdit object.IOException
- if an error occurred on the coprocessorpublic Result preAppendAfterRowLock(Append append) throws IOException
append
- append objectIOException
- if an error occurred on the coprocessorpublic Result preIncrement(Increment increment, WALEdit edit) throws IOException
increment
- increment objectedit
- The WALEdit object.IOException
- if an error occurred on the coprocessorpublic Result preIncrementAfterRowLock(Increment increment) throws IOException
increment
- increment objectIOException
- if an error occurred on the coprocessorpublic Result postAppend(Append append, Result result, WALEdit edit) throws IOException
append
- Append objectresult
- the result returned by the appendedit
- The WALEdit object.IOException
- if an error occurred on the coprocessorpublic Result postIncrement(Increment increment, Result result, WALEdit edit) throws IOException
increment
- increment objectresult
- the result returned by postIncrementedit
- The WALEdit object.IOException
- if an error occurred on the coprocessorpublic void preScannerOpen(Scan scan) throws IOException
scan
- the Scan specificationIOException
- Exceptionpublic RegionScanner postScannerOpen(Scan scan, RegionScanner s) throws IOException
scan
- the Scan specifications
- the scannerIOException
- Exceptionpublic Boolean preScannerNext(InternalScanner s, List<Result> results, int limit) throws IOException
s
- the scannerresults
- the result set returned by the region serverlimit
- the maximum number of results to returnIOException
- Exceptionpublic boolean postScannerNext(InternalScanner s, List<Result> results, int limit, boolean hasMore) throws IOException
s
- the scannerresults
- the result set returned by the region serverlimit
- the maximum number of results to return n * @return 'has more' indication to
give to clientIOException
- Exceptionpublic boolean postScannerFilterRow(InternalScanner s, Cell curRowCell) throws IOException
s
- the scannercurRowCell
- The cell in the current row which got filtered outIOException
public boolean preScannerClose(InternalScanner s) throws IOException
s
- the scannerIOException
- Exceptionpublic void postScannerClose(InternalScanner s) throws IOException
IOException
- Exceptionpublic ScanInfo preStoreScannerOpen(HStore store, Scan scan) throws IOException
IOException
public void preReplayWALs(RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException
info
- the RegionInfo for this regionedits
- the file of recovered editsIOException
public void postReplayWALs(RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException
info
- the RegionInfo for this regionedits
- the file of recovered editsIOException
- Exception@Deprecated public boolean preWALRestore(RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
IOException
@Deprecated public void postWALRestore(RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
IOException
public void preBulkLoadHFile(List<Pair<byte[],String>> familyPaths) throws IOException
familyPaths
- pairs of { CF, file path } submitted for bulk loadIOException
public boolean preCommitStoreFile(byte[] family, List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs) throws IOException
IOException
public void postCommitStoreFile(byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) throws IOException
IOException
public void postBulkLoadHFile(List<Pair<byte[],String>> familyPaths, Map<byte[],List<org.apache.hadoop.fs.Path>> map) throws IOException
familyPaths
- pairs of { CF, file path } submitted for bulk loadmap
- Map of CF to List of file paths for the final loaded files nIOException
public void postStartRegionOperation(Region.Operation op) throws IOException
IOException
public void postCloseRegionOperation(Region.Operation op) throws IOException
IOException
public StoreFileReader preStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r) throws IOException
fs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- Full size of the file n * @param r original reference file. This will be not null
only when reading a split file.IOException
public StoreFileReader postStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader reader) throws IOException
fs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- Full size of the file n * @param r original reference file. This will be not null
only when reading a split file.reader
- the base reader instanceIOException
public List<Pair<Cell,Cell>> postIncrementBeforeWAL(Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
IOException
public List<Pair<Cell,Cell>> postAppendBeforeWAL(Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
IOException
public void preWALAppend(WALKey key, WALEdit edit) throws IOException
IOException
public com.google.protobuf.Message preEndpointInvocation(com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request) throws IOException
IOException
public void postEndpointInvocation(com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request, com.google.protobuf.Message.Builder responseBuilder) throws IOException
IOException
@Deprecated public DeleteTracker postInstantiateDeleteTracker(DeleteTracker result) throws IOException
IOException
public void prePrepareBulkLoad(User user) throws IOException
IOException
public void preCleanupBulkLoad(User user) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.