Class RegionCoprocessorHost
java.lang.Object
org.apache.hadoop.hbase.coprocessor.CoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost
@Private
public class RegionCoprocessorHost
extends CoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
Implements the coprocessor environment and runtime support for coprocessors loaded within a
Region.-
Nested Class Summary
Nested ClassesModifier and TypeClassDescription(package private) classprivate static classEncapsulation of the environment of each coprocessorprivate static classSpecial version of RegionEnvironment that exposes RegionServerServices for Core Coprocessors only.(package private) class(package private) static classNested classes/interfaces inherited from class org.apache.hadoop.hbase.coprocessor.CoprocessorHost
CoprocessorHost.ObserverGetter<C,O>, CoprocessorHost.ObserverOperationWithoutResult<O>, CoprocessorHost.ObserverOperationWithResult<O, R> -
Field Summary
FieldsModifier and TypeFieldDescriptionprivate final booleanprivate static final org.slf4j.Logger(package private) HRegionThe region(package private) RegionServerServicesThe region server servicesprivate static final org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap<String,ConcurrentMap<String, Object>> Fields inherited from class org.apache.hadoop.hbase.coprocessor.CoprocessorHost
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, RPC_COPROCESSOR_CONF_KEY, SKIP_LOAD_DUPLICATE_TABLE_COPROCESSOR, USER_COPROCESSORS_ENABLED_CONF_KEY, USER_REGION_COPROCESSOR_CONF_KEY, WAL_COPROCESSOR_CONF_KEY -
Constructor Summary
ConstructorsConstructorDescriptionRegionCoprocessorHost(HRegion region, RegionServerServices rsServices, org.apache.hadoop.conf.Configuration conf) Constructor -
Method Summary
Modifier and TypeMethodDescriptioncheckAndGetInstance(Class<?> implClass) Called when a new Coprocessor class needs to be loaded.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(package private) voidloadTableCoprocessors(org.apache.hadoop.conf.Configuration conf) postAppend(Append append, Result result, WALEdit edit) voidpostBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) voidpostBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) voidpostBulkLoadHFile(List<Pair<byte[], String>> familyPaths, Map<byte[], List<org.apache.hadoop.fs.Path>> map) postCheckAndMutate(CheckAndMutate checkAndMutate, CheckAndMutateResult result) voidpostClose(boolean abortRequested) Invoked after a region is closedvoidvoidpostCommitStoreFile(byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) voidpostCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) Called after the store compaction has completed.voidpostCompactSelection(HStore store, List<HStoreFile> selected, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) Called after theHStoreFiles to be compacted have been selected from the available candidates.voidpostDelete(Delete delete, WALEdit edit) voidpostEndpointInvocation(org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request, org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder responseBuilder) booleanpostExists(Get get, boolean result) voidpostFlush(FlushLifeCycleTracker tracker) Invoked after a memstore flushvoidpostFlush(HStore store, HStoreFile storeFile, FlushLifeCycleTracker tracker) Invoked after a memstore flushvoidpostIncrement(Increment increment, Result result, WALEdit edit) voidpostMemStoreCompaction(HStore store) Invoked after in memory compaction.voidpostOpen()Invoked after a region openvoidvoidpostReplayWALs(RegionInfo info, org.apache.hadoop.fs.Path edits) voidbooleanpostScannerFilterRow(InternalScanner s, Cell curRowCell) This will be called by the scan flow when the current scanned row is being filtered out by the filter.booleanpostScannerNext(InternalScanner s, List<Result> results, int limit, boolean hasMore) postScannerOpen(Scan scan, RegionScanner s) voidpostStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader reader) voidpostWALRestore(RegionInfo info, WALKey logKey, WALEdit logEdit) Supports Coprocessor 'bypass'.preAppendAfterRowLock(Append append) Supports Coprocessor 'bypass'.voidpreBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) voidpreBulkLoadHFile(List<Pair<byte[], String>> familyPaths) preCheckAndMutate(CheckAndMutate checkAndMutate) Supports Coprocessor 'bypass'.preCheckAndMutateAfterRowLock(CheckAndMutate checkAndMutate) Supports Coprocessor 'bypass'.voidpreCleanupBulkLoad(User user) voidpreClose(boolean abortRequested) Invoked before a region is closedbooleanpreCommitStoreFile(byte[] family, List<Pair<org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path>> pairs) preCompact(HStore store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) Called prior to rewriting the store files selected for compactionpreCompactScannerOpen(HStore store, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) Called prior to opening store scanner for compaction.booleanpreCompactSelection(HStore store, List<HStoreFile> candidates, CompactionLifeCycleTracker tracker, User user) Called prior to selecting theHStoreFiles for compaction from the list of currently available candidates.booleanSupports Coprocessor 'bypass'.org.apache.hbase.thirdparty.com.google.protobuf.MessagepreEndpointInvocation(org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request) Supports Coprocessor 'bypass'.voidpreFlush(FlushLifeCycleTracker tracker) Invoked before a memstore flushpreFlush(HStore store, InternalScanner scanner, FlushLifeCycleTracker tracker) Invoked before a memstore flushpreFlushScannerOpen(HStore store, FlushLifeCycleTracker tracker) Invoked before create StoreScanner for flush.booleanSupports Coprocessor 'bypass'.preIncrement(Increment increment, WALEdit edit) Supports Coprocessor 'bypass'.preIncrementAfterRowLock(Increment increment) Supports Coprocessor 'bypass'.voidpreMemStoreCompaction(HStore store) Invoked before in memory compaction.preMemStoreCompactionCompact(HStore store, InternalScanner scanner) Invoked before compacting memstore.Invoked before create StoreScanner for in memory compaction.voidpreOpen()Invoked before a region open.voidprePrepareBulkLoad(User user) booleanprePrepareTimeStampForDeleteVersion(Mutation mutation, Cell kv, byte[] byteNow, Get get) Supports Coprocessor 'bypass'.booleanSupports Coprocessor 'bypass'.voidpreReplayWALs(RegionInfo info, org.apache.hadoop.fs.Path edits) booleanSupports Coprocessor 'bypass'.preScannerNext(InternalScanner s, List<Result> results, int limit) voidpreScannerOpen(Scan scan) preStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r) preStoreScannerOpen(HStore store, Scan scan) Called before open store scanner for user scan.voidpreWALAppend(WALKey key, WALEdit edit) booleanpreWALRestore(RegionInfo info, WALKey logKey, WALEdit logEdit) Supports Coprocessor 'bypass'.static voidtestTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf, TableDescriptor htd) Sanity check the table coprocessor attributes of the supplied schema.Methods inherited from class org.apache.hadoop.hbase.coprocessor.CoprocessorHost
abortServer, abortServer, checkAndLoadInstance, execOperation, execOperationWithResult, execShutdown, findCoprocessor, findCoprocessor, findCoprocessorEnvironment, findCoprocessors, getCoprocessors, getLoadedCoprocessors, handleCoprocessorThrowable, load, load, load, loadSystemCoprocessors, shutdown
-
Field Details
-
LOG
-
SHARED_DATA_MAP
private static final org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap<String,ConcurrentMap<String, SHARED_DATA_MAPObject>> -
hasCustomPostScannerFilterRow
-
rsServices
The region server services -
region
The region -
regionObserverGetter
-
endpointObserverGetter
-
-
Constructor Details
-
RegionCoprocessorHost
public RegionCoprocessorHost(HRegion region, RegionServerServices rsServices, org.apache.hadoop.conf.Configuration conf) Constructor- Parameters:
region- the regionrsServices- interface to available region server functionalityconf- the configuration
-
-
Method Details
-
hasCustomPostScannerFilterRow
-
getTableCoprocessorAttrsFromSchema
static List<RegionCoprocessorHost.TableCoprocessorAttribute> getTableCoprocessorAttrsFromSchema(org.apache.hadoop.conf.Configuration conf, TableDescriptor htd) -
testTableCoprocessorAttrs
public static void testTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf, TableDescriptor htd) throws IOException Sanity check the table coprocessor attributes of the supplied schema. Will throw an exception if there is a problem.- Throws:
IOException
-
loadTableCoprocessors
-
createEnvironment
public RegionCoprocessorHost.RegionEnvironment createEnvironment(RegionCoprocessor instance, int priority, int seq, org.apache.hadoop.conf.Configuration conf) Description copied from class:CoprocessorHostCalled when a new Coprocessor class is loaded- Specified by:
createEnvironmentin classCoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment>
-
checkAndGetInstance
public RegionCoprocessor checkAndGetInstance(Class<?> implClass) throws InstantiationException, IllegalAccessException Description copied from class:CoprocessorHostCalled when a new Coprocessor class needs to be loaded. Checks if type of the given class is what the corresponding host implementation expects. If it is of correct type, returns an instance of the coprocessor to be loaded. If not, returns null. If an exception occurs when trying to create instance of a coprocessor, it's passed up and eventually results into server aborting.- Specified by:
checkAndGetInstancein classCoprocessorHost<RegionCoprocessor,RegionCoprocessorEnvironment> - Throws:
InstantiationExceptionIllegalAccessException
-
preOpen
Invoked before a region open.- Throws:
IOException- Signals that an I/O exception has occurred.
-
postOpen
Invoked after a region open -
preClose
Invoked before a region is closed- Parameters:
abortRequested- true if the server is aborting- Throws:
IOException
-
postClose
Invoked after a region is closed- Parameters:
abortRequested- true if the server is aborting
-
preCompactSelection
public boolean preCompactSelection(HStore store, List<HStoreFile> candidates, CompactionLifeCycleTracker tracker, User user) throws IOException Called prior to selecting theHStoreFiles 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.- Parameters:
store- The store where compaction is being requestedcandidates- The currently available store filestracker- used to track the life cycle of a compactionuser- the user- Throws:
IOException
-
postCompactSelection
public void postCompactSelection(HStore store, List<HStoreFile> selected, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException Called after theHStoreFiles to be compacted have been selected from the available candidates.- Parameters:
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 user- Throws:
IOException
-
preCompactScannerOpen
public ScanInfo preCompactScannerOpen(HStore store, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException Called prior to opening store scanner for compaction.- Throws:
IOException
-
preCompact
public InternalScanner preCompact(HStore store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException Called prior to rewriting the store files selected for compaction- Parameters:
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 user- Returns:
- Scanner to use (cannot be null!)
- Throws:
IOException
-
postCompact
public void postCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException Called after the store compaction has completed.- Parameters:
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- Throws:
IOException
-
preFlushScannerOpen
Invoked before create StoreScanner for flush.- Throws:
IOException
-
preFlush
public InternalScanner preFlush(HStore store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException Invoked before a memstore flush- Returns:
- Scanner to use (cannot be null!)
- Throws:
IOException
-
preFlush
Invoked before a memstore flush- Throws:
IOException
-
postFlush
Invoked after a memstore flush- Throws:
IOException
-
preMemStoreCompaction
Invoked before in memory compaction.- Throws:
IOException
-
preMemStoreCompactionCompactScannerOpen
Invoked before create StoreScanner for in memory compaction.- Throws:
IOException
-
preMemStoreCompactionCompact
public InternalScanner preMemStoreCompactionCompact(HStore store, InternalScanner scanner) throws IOException Invoked before compacting memstore.- Throws:
IOException
-
postMemStoreCompaction
Invoked after in memory compaction.- Throws:
IOException
-
postFlush
public void postFlush(HStore store, HStoreFile storeFile, FlushLifeCycleTracker tracker) throws IOException Invoked after a memstore flush- Throws:
IOException
-
preGet
Supports Coprocessor 'bypass'.- Parameters:
get- the Get requestresults- What to return if return is true/'bypass'.- Returns:
- true if default processing should be bypassed.
- Throws:
IOException- Exception
-
postGet
- Parameters:
get- the Get requestresults- the result set- Throws:
IOException- Exception
-
preExists
Supports Coprocessor 'bypass'.- Parameters:
get- the Get request- Returns:
- true or false to return to client if bypassing normal operation, or null otherwise
- Throws:
IOException- Exception
-
postExists
- Parameters:
get- the Get requestresult- the result returned by the region server- Returns:
- the result to return to the client
- Throws:
IOException- Exception
-
prePut
Supports Coprocessor 'bypass'.- Parameters:
put- The Put objectedit- The WALEdit object.- Returns:
- true if default processing should be bypassed
- Throws:
IOException- Exception
-
prePrepareTimeStampForDeleteVersion
public boolean prePrepareTimeStampForDeleteVersion(Mutation mutation, Cell kv, byte[] byteNow, Get get) throws IOException Supports Coprocessor 'bypass'.- Parameters:
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 used- Returns:
- true if default processing should be bypassed
- Throws:
IOException
-
postPut
- Parameters:
put- The Put objectedit- The WALEdit object.- Throws:
IOException- Exception
-
preDelete
Supports Coprocessor 'bypass'.- Parameters:
delete- The Delete objectedit- The WALEdit object.- Returns:
- true if default processing should be bypassed
- Throws:
IOException- Exception
-
postDelete
- Parameters:
delete- The Delete objectedit- The WALEdit object.- Throws:
IOException- Exception
-
preBatchMutate
- Throws:
IOException
-
postBatchMutate
- Throws:
IOException
-
postBatchMutateIndispensably
public void postBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException - Throws:
IOException
-
preCheckAndMutate
Supports Coprocessor 'bypass'.- Parameters:
checkAndMutate- the CheckAndMutate object- Returns:
- true or false to return to client if default processing should be bypassed, or null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
preCheckAndMutateAfterRowLock
public CheckAndMutateResult preCheckAndMutateAfterRowLock(CheckAndMutate checkAndMutate) throws IOException Supports Coprocessor 'bypass'.- Parameters:
checkAndMutate- the CheckAndMutate object- Returns:
- true or false to return to client if default processing should be bypassed, or null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
postCheckAndMutate
public CheckAndMutateResult postCheckAndMutate(CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException - Parameters:
checkAndMutate- the CheckAndMutate objectresult- the result returned by the checkAndMutate- Returns:
- true or false to return to client if default processing should be bypassed, or null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
preAppend
Supports Coprocessor 'bypass'.- Parameters:
append- append objectedit- The WALEdit object.- Returns:
- result to return to client if default operation should be bypassed, null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
preAppendAfterRowLock
Supports Coprocessor 'bypass'.- Parameters:
append- append object- Returns:
- result to return to client if default operation should be bypassed, null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
preIncrement
Supports Coprocessor 'bypass'.- Parameters:
increment- increment objectedit- The WALEdit object.- Returns:
- result to return to client if default operation should be bypassed, null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
preIncrementAfterRowLock
Supports Coprocessor 'bypass'.- Parameters:
increment- increment object- Returns:
- result to return to client if default operation should be bypassed, null otherwise
- Throws:
IOException- if an error occurred on the coprocessor
-
postAppend
- Parameters:
append- Append objectresult- the result returned by the appendedit- The WALEdit object.- Throws:
IOException- if an error occurred on the coprocessor
-
postIncrement
- Parameters:
increment- increment objectresult- the result returned by postIncrementedit- The WALEdit object.- Throws:
IOException- if an error occurred on the coprocessor
-
preScannerOpen
- Parameters:
scan- the Scan specification- Throws:
IOException- Exception
-
postScannerOpen
- Parameters:
scan- the Scan specifications- the scanner- Returns:
- the scanner instance to use
- Throws:
IOException- Exception
-
preScannerNext
public Boolean preScannerNext(InternalScanner s, List<Result> results, int limit) throws IOException - Parameters:
s- the scannerresults- the result set returned by the region serverlimit- the maximum number of results to return- Returns:
- 'has next' indication to client if bypassing default behavior, or null otherwise
- Throws:
IOException- Exception
-
postScannerNext
public boolean postScannerNext(InternalScanner s, List<Result> results, int limit, boolean hasMore) throws IOException - Parameters:
s- the scannerresults- the result set returned by the region serverlimit- the maximum number of results to return- Returns:
- 'has more' indication to give to client
- Throws:
IOException- Exception
-
postScannerFilterRow
This will be called by the scan flow when the current scanned row is being filtered out by the filter.- Parameters:
s- the scannercurRowCell- The cell in the current row which got filtered out- Returns:
- whether more rows are available for the scanner or not
- Throws:
IOException
-
preScannerClose
Supports Coprocessor 'bypass'.- Parameters:
s- the scanner- Returns:
- true if default behavior should be bypassed, false otherwise
- Throws:
IOException- Exception
-
postScannerClose
- Throws:
IOException- Exception
-
preStoreScannerOpen
Called before open store scanner for user scan.- Throws:
IOException
-
preReplayWALs
- Parameters:
info- the RegionInfo for this regionedits- the file of recovered edits- Throws:
IOException
-
postReplayWALs
- Parameters:
info- the RegionInfo for this regionedits- the file of recovered edits- Throws:
IOException- Exception
-
preWALRestore
Supports Coprocessor 'bypass'.- Returns:
- true if default behavior should be bypassed, false otherwise
- Throws:
IOException
-
postWALRestore
- Throws:
IOException
-
preBulkLoadHFile
- Parameters:
familyPaths- pairs of { CF, file path } submitted for bulk load- Throws:
IOException
-
preCommitStoreFile
public boolean preCommitStoreFile(byte[] family, List<Pair<org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path>> pairs) throws IOException- Throws:
IOException
-
postCommitStoreFile
public void postCommitStoreFile(byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) throws IOException - Throws:
IOException
-
postBulkLoadHFile
public void postBulkLoadHFile(List<Pair<byte[], String>> familyPaths, Map<byte[], throws IOExceptionList<org.apache.hadoop.fs.Path>> map) - Parameters:
familyPaths- pairs of { CF, file path } submitted for bulk loadmap- Map of CF to List of file paths for the final loaded files- Throws:
IOException
-
postStartRegionOperation
- Throws:
IOException
-
postCloseRegionOperation
- Throws:
IOException
-
preStoreFileReaderOpen
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 - Parameters:
fs- 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.- Returns:
- a Reader instance to use instead of the base reader if overriding default behavior, null otherwise
- Throws:
IOException
-
postStoreFileReaderOpen
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 - Parameters:
fs- 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 instance- Returns:
- The reader to use
- Throws:
IOException
-
postIncrementBeforeWAL
public List<Pair<Cell,Cell>> postIncrementBeforeWAL(Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOException- Throws:
IOException
-
postAppendBeforeWAL
public List<Pair<Cell,Cell>> postAppendBeforeWAL(Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOException- Throws:
IOException
-
preWALAppend
- Throws:
IOException
-
preEndpointInvocation
public org.apache.hbase.thirdparty.com.google.protobuf.Message preEndpointInvocation(org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request) throws IOException - Throws:
IOException
-
postEndpointInvocation
public void postEndpointInvocation(org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request, org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder responseBuilder) throws IOException - Throws:
IOException
-
postInstantiateDeleteTracker
- Throws:
IOException
-
prePrepareBulkLoad
- Throws:
IOException
-
preCleanupBulkLoad
- Throws:
IOException
-