public interface RpcService
RpcEndpoint
.
Connecting to a rpc server will return a RpcGateway
which can be used to call remote
procedures.Modifier and Type | Method and Description |
---|---|
<C extends RpcGateway> |
connect(String address,
Class<C> clazz)
Connect to a remote rpc server under the provided address.
|
<F extends Serializable,C extends FencedRpcGateway<F>> |
connect(String address,
F fencingToken,
Class<C> clazz)
Connect to a remote fenced rpc server under the provided address.
|
<T> CompletableFuture<T> |
execute(Callable<T> callable)
Execute the given callable and return its result as a
CompletableFuture . |
void |
execute(Runnable runnable)
Execute the given runnable in the executor of the RPC service.
|
<F extends Serializable> |
fenceRpcServer(RpcServer rpcServer,
F fencingToken)
Fence the given RpcServer with the given fencing token.
|
String |
getAddress()
Return the hostname or host address under which the rpc service can be reached.
|
Executor |
getExecutor()
Gets the executor, provided by this RPC service.
|
int |
getPort()
Return the port under which the rpc service is reachable.
|
ScheduledExecutor |
getScheduledExecutor()
Gets a scheduled executor from the RPC service.
|
CompletableFuture<Void> |
getTerminationFuture()
Returns a future indicating when the RPC service has been shut down.
|
ScheduledFuture<?> |
scheduleRunnable(Runnable runnable,
long delay,
TimeUnit unit)
Execute the runnable in the execution context of this RPC Service, as returned by
getExecutor() , after a scheduled delay. |
<C extends RpcEndpoint & RpcGateway> |
startServer(C rpcEndpoint)
Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint.
|
void |
stopServer(RpcServer selfGateway)
Stop the underlying rpc server of the provided self gateway.
|
CompletableFuture<Void> |
stopService()
Trigger the asynchronous stopping of the
RpcService . |
String getAddress()
int getPort()
<C extends RpcGateway> CompletableFuture<C> connect(String address, Class<C> clazz)
RpcConnectionException
.C
- Type of the rpc gateway to returnaddress
- Address of the remote rpc serverclazz
- Class of the rpc gateway to returnRpcConnectionException
if the
connection attempt failed<F extends Serializable,C extends FencedRpcGateway<F>> CompletableFuture<C> connect(String address, F fencingToken, Class<C> clazz)
RpcConnectionException
.F
- Type of the fencing tokenC
- Type of the rpc gateway to returnaddress
- Address of the remote rpc serverfencingToken
- Fencing token to be used when communicating with the serverclazz
- Class of the rpc gateway to returnRpcConnectionException
if the
connection attempt failed<C extends RpcEndpoint & RpcGateway> RpcServer startServer(C rpcEndpoint)
C
- Type of the rpc endpointrpcEndpoint
- Rpc protocol to dispatch the rpcs to<F extends Serializable> RpcServer fenceRpcServer(RpcServer rpcServer, F fencingToken)
Fencing the RpcServer means that we fix the fencing token to the provided value.
All RPCs will then be enriched with this fencing token. This expects that the receiving
RPC endpoint extends FencedRpcEndpoint
.
F
- type of the fencing tokenrpcServer
- to fence with the given fencing tokenfencingToken
- to fence the RpcServer withvoid stopServer(RpcServer selfGateway)
selfGateway
- Self gateway describing the underlying rpc serverCompletableFuture<Void> stopService()
RpcService
.RpcService
has been
fully stopped.CompletableFuture<Void> getTerminationFuture()
Executor getExecutor()
handleAsync(...)
or thenAcceptAsync(...)
methods of futures.
IMPORTANT: This executor does not isolate the method invocations against
any concurrent invocations and is therefore not suitable to run completion methods of futures
that modify state of an RpcEndpoint
. For such operations, one needs to use the
MainThreadExecutionContext
of that
RpcEndpoint
.
ScheduledExecutor getScheduledExecutor()
IMPORTANT: This executor does not isolate the method invocations against
any concurrent invocations and is therefore not suitable to run completion methods of futures
that modify state of an RpcEndpoint
. For such operations, one needs to use the
MainThreadExecutionContext
of that
RpcEndpoint
.
ScheduledFuture<?> scheduleRunnable(Runnable runnable, long delay, TimeUnit unit)
getExecutor()
, after a scheduled delay.runnable
- Runnable to be executeddelay
- The delay after which the runnable will be executedvoid execute(Runnable runnable)
RpcEndpoint
.
IMPORTANT: This executor does not isolate the method invocations against
any concurrent invocations and is therefore not suitable to run completion methods of futures
that modify state of an RpcEndpoint
. For such operations, one needs to use the
MainThreadExecutionContext
of that
RpcEndpoint
.
runnable
- to execute<T> CompletableFuture<T> execute(Callable<T> callable)
CompletableFuture
. This method can be used
to run code outside of the main thread of a RpcEndpoint
.
IMPORTANT: This executor does not isolate the method invocations against
any concurrent invocations and is therefore not suitable to run completion methods of futures
that modify state of an RpcEndpoint
. For such operations, one needs to use the
MainThreadExecutionContext
of that
RpcEndpoint
.
T
- is the return value typecallable
- to executeCopyright © 2014–2020 The Apache Software Foundation. All rights reserved.