@InterfaceAudience.LimitedPrivate(value="Replication") public interface ReplicationEndpoint extends ReplicationPeerConfigListener
ReplicationPeerConfig
. A ReplicationEndpoint is run in a thread
in each region server in the same process.
ReplicationEndpoint is closely tied to ReplicationSource in a producer-consumer relation. ReplicationSource is an HBase-private class which tails the logs and manages the queue of logs plus management and persistence of all the state for replication. ReplicationEndpoint on the other hand is responsible for doing the actual shipping and persisting of the WAL entries in the other cluster.
Modifier and Type | Interface and Description |
---|---|
static class |
ReplicationEndpoint.Context |
static class |
ReplicationEndpoint.ReplicateContext
A context for
replicate(ReplicateContext) method. |
Modifier and Type | Method and Description |
---|---|
void |
awaitRunning()
Waits for the
ReplicationEndpoint to be up and running. |
void |
awaitRunning(long timeout,
TimeUnit unit)
Waits for the
ReplicationEndpoint to to be up and running for no more
than the given time. |
void |
awaitTerminated()
Waits for the
ReplicationEndpoint to reach the terminated (internal) state. |
void |
awaitTerminated(long timeout,
TimeUnit unit)
Waits for the
ReplicationEndpoint to reach a terminal state for no
more than the given time. |
boolean |
canReplicateToSameCluster()
Whether or not, the replication endpoint can replicate to it's source cluster with the same
UUID
|
Throwable |
failureCause()
Returns the
Throwable that caused this service to fail. |
UUID |
getPeerUUID()
Returns a UUID of the provided peer id.
|
WALEntryFilter |
getWALEntryfilter()
Returns a WALEntryFilter to use for filtering out WALEntries from the log.
|
void |
init(ReplicationEndpoint.Context context)
Initialize the replication endpoint with the given context.
|
boolean |
isRunning()
Returns
true if this service is RUNNING. |
boolean |
isStarting() |
boolean |
replicate(ReplicationEndpoint.ReplicateContext replicateContext)
Replicate the given set of entries (in the context) to the other cluster.
|
void |
start()
Initiates service startup and returns immediately.
|
void |
stop()
If the service is starting or running, this initiates service shutdown and returns immediately.
|
peerConfigUpdated
void init(ReplicationEndpoint.Context context) throws IOException
context
- replication contextIOException
boolean canReplicateToSameCluster()
UUID getPeerUUID()
WALEntryFilter getWALEntryfilter()
WALEntryFilter
or null.boolean replicate(ReplicationEndpoint.ReplicateContext replicateContext)
replicateContext
- a context where WAL entries and other
parameters can be obtained.boolean isRunning()
true
if this service is RUNNING.boolean isStarting()
true
is this service is STARTING (but not yet RUNNING).void start()
IllegalStateException
- if the service is not new, if it has been run already.void awaitRunning()
ReplicationEndpoint
to be up and running.IllegalStateException
- if the service reaches a state from which it is not possible to
enter the (internal) running state. e.g. if the state is terminated when this method is
called then this will throw an IllegalStateException.void awaitRunning(long timeout, TimeUnit unit) throws TimeoutException
ReplicationEndpoint
to to be up and running for no more
than the given time.timeout
- the maximum time to waitunit
- the time unit of the timeout argumentTimeoutException
- if the service has not reached the given state within the deadlineIllegalStateException
- if the service reaches a state from which it is not possible to
enter the (internal) running state. e.g. if the state is terminated when this method is
called then this will throw an IllegalStateException.void stop()
void awaitTerminated()
ReplicationEndpoint
to reach the terminated (internal) state.IllegalStateException
- if the service FAILED.void awaitTerminated(long timeout, TimeUnit unit) throws TimeoutException
ReplicationEndpoint
to reach a terminal state for no
more than the given time.timeout
- the maximum time to waitunit
- the time unit of the timeout argumentTimeoutException
- if the service has not reached the given state within the deadlineIllegalStateException
- if the service FAILED.Throwable failureCause()
Throwable
that caused this service to fail.IllegalStateException
- if this service's state isn't FAILED.Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.