F
- type of the fencing tokenpublic abstract class FencedRpcEndpoint<F extends Serializable> extends RpcEndpoint
RpcEndpoint
. A fenced rpc endpoint expects all rpc messages being
enriched with fencing tokens. Furthermore, the rpc endpoint has its own fencing token assigned.
The rpc is then only executed if the attached fencing token equals the endpoint's own token.RpcEndpoint.MainThreadExecutor
log, rpcServer
Modifier | Constructor and Description |
---|---|
protected |
FencedRpcEndpoint(RpcService rpcService,
F fencingToken) |
protected |
FencedRpcEndpoint(RpcService rpcService,
String endpointId,
F fencingToken) |
Modifier and Type | Method and Description |
---|---|
protected <V> CompletableFuture<V> |
callAsyncWithoutFencing(Callable<V> callable,
Time timeout)
Run the given callable in the main thread of the RpcEndpoint without checking the fencing
token.
|
F |
getFencingToken() |
protected RpcEndpoint.MainThreadExecutor |
getMainThreadExecutor()
Returns a main thread executor which is bound to the currently valid fencing token.
|
protected Executor |
getUnfencedMainThreadExecutor()
Returns a main thread executor which is not bound to the fencing token.
|
protected void |
runAsyncWithoutFencing(Runnable runnable)
Run the given runnable in the main thread of the RpcEndpoint without checking the fencing
token.
|
protected void |
setFencingToken(F newFencingToken) |
callAsync, closeAsync, getAddress, getEndpointId, getHostname, getRpcService, getSelfGateway, getTerminationFuture, internalCallOnStart, internalCallOnStop, isRunning, onStart, onStop, runAsync, scheduleRunAsync, scheduleRunAsync, start, stop, validateRunsInMainThread
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
close
protected FencedRpcEndpoint(RpcService rpcService, String endpointId, @Nullable F fencingToken)
protected FencedRpcEndpoint(RpcService rpcService, @Nullable F fencingToken)
public F getFencingToken()
protected RpcEndpoint.MainThreadExecutor getMainThreadExecutor()
getMainThreadExecutor
in class RpcEndpoint
protected Executor getUnfencedMainThreadExecutor()
Runnable
which are executed with this executor will always be executed.protected void runAsyncWithoutFencing(Runnable runnable)
runnable
- to execute in the main thread of the rpc endpoint without checking the
fencing token.protected <V> CompletableFuture<V> callAsyncWithoutFencing(Callable<V> callable, Time timeout)
callable
- to run in the main thread of the rpc endpoint without checkint the fencing
token.timeout
- for the operation.Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.