Flink RPC通信流程解析
本文档主要介绍 Flink RPC 通信的核心组件与实现机制,包括 RpcSystem、RpcService、RpcEndpoint、RpcGateway、InvocationHandler,以及与其强相关的工具/实现类(RpcUtils、PekkoRpcServiceUtils、RobustActorSystem、SupervisorActor、StartRpcActor、PekkoRpcActor)的职责与交互关系。
一、Pekko(Akka) 前置知识(本地 demo)
Flink 的 RPC 底座来自 Akka 的分支 Apache Pekko (接口层仍叫 flink-rpc-akka)。如果你对 Actor 模型不熟,先用下面的最小 demo 建立直觉:ActorSystem/Actor 的定义、tell/ask 的区别、以及"本机两个 ActorSystem 通过 remoting 通信"的基本形态。
1.1 核心概念速览
- ActorSystem :Actor 的容器与运行时,负责线程模型(dispatcher)、调度器(scheduler)、序列化/网络(remoting)等全局能力;Actor 的路径空间从
/user开始。 - Actor(AbstractActor) :消息处理单元;通过
createReceive()声明"收到某类消息要做什么"。 - ActorRef / ActorSelection :
ActorRef是强引用(本地创建/拿到);ActorSelection是按路径查找(可跨进程/跨 ActorSystem)。 - Props :Actor 的"构造描述",用于
actorSystem.actorOf(Props.create(XxxActor.class), name)。 - tell vs ask :
tell单向发送(不等结果);ask请求-响应(返回CompletionStage),更接近 Flink RPC 的"异步方法返回值"语义。
1.2 Demo:JDK 动态代理(Proxy.newProxyInstance)
Flink 的 RpcGateway 不是某个固定实现类,而是运行时用 Proxy.newProxyInstance(...) 生成的代理对象;当你调用网关方法时,本质会进入 InvocationHandler.invoke(...)(Flink 中对应 PekkoInvocationHandler),再由它决定是本地处理还是把调用转成消息发给 Actor。
Proxy.newProxyInstance(loader, interfaces, handler)的含义:loader:生成出来的代理类要挂在哪个ClassLoader下;通常选接口的 classloader(EchoGateway.class.getClassLoader())。interfaces:这个代理"要实现哪些接口"。这里的new Class<?>[] {EchoGateway.class}表示:生成的代理对象gateway在类型系统上就是一个EchoGateway,因此你可以强转为EchoGateway并调用echo(...)。handler:方法拦截器;只要调用的是interfaces中声明的方法,最终都会回到handler.invoke(proxy, method, args)。
interfaces必须都是 interface (不能是普通 class):- 传入普通类会直接失败,典型报错是
IllegalArgumentException: xxx is not an interface。 - 这是因为 JDK 动态代理的机制是 "生成一个新类 extends Proxy,并 implements 这些接口",它不会去继承你的业务类。
- 传入普通类会直接失败,典型报错是
- 可以传多个接口:
new Class<?>[] {EchoGateway.class, Closeable.class},这样代理对象同时也是多个接口类型;Flink 场景里常见是同时包含RpcGateway、业务*Gateway,以及框架侧需要的若干接口。
java
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.util.concurrent.CompletableFuture;
public class DynamicProxyDemo {
interface EchoGateway {
CompletableFuture<String> echo(String msg);
}
static final class EchoInvocationHandler implements InvocationHandler {
@Override
public Object invoke(Object proxy, Method method, Object[] args) {
if (method.getDeclaringClass() == Object.class) {
return switch (method.getName()) {
case "toString" -> "EchoGatewayProxy";
case "hashCode" -> System.identityHashCode(proxy);
case "equals" -> proxy == args[0];
default -> throw new UnsupportedOperationException(method.getName());
};
}
if (method.getName().equals("echo")) {
String msg = (String) args[0];
return CompletableFuture.completedFuture("echo:" + msg);
}
throw new UnsupportedOperationException(method.toGenericString());
}
}
public static void main(String[] args) {
EchoGateway gateway =
(EchoGateway)
Proxy.newProxyInstance(
EchoGateway.class.getClassLoader(),
new Class<?>[] {EchoGateway.class},
new EchoInvocationHandler());
gateway.echo("ping").thenAccept(System.out::println).join();
}
}
如果把 interfaces 传成"类"会怎样(示例):
java
import java.lang.reflect.Proxy;
public class ProxyBadInterfacesDemo {
static final class EchoService {
String echo(String msg) {
return "echo:" + msg;
}
}
public static void main(String[] args) {
Object o =
Proxy.newProxyInstance(
EchoService.class.getClassLoader(),
new Class<?>[] {EchoService.class},
(p, m, a) -> null);
}
}
运行会抛 IllegalArgumentException(因为 EchoService 不是 interface)。如果你确实需要"代理一个类"(子类化),那就是另一条技术路线(通常是字节码生成/子类代理),而 Flink 的 RpcGateway 选择的是 JDK 动态代理这条"接口代理"路径。
1.3 Demo 1:同一个 ActorSystem 内的"客户端/服务端"通信
java
import org.apache.pekko.actor.AbstractActor;
import org.apache.pekko.actor.ActorRef;
import org.apache.pekko.actor.ActorSystem;
import org.apache.pekko.actor.Props;
import org.apache.pekko.japi.pf.ReceiveBuilder;
import org.apache.pekko.pattern.Patterns;
import java.time.Duration;
import java.util.concurrent.CompletionStage;
public class LocalDemo {
public static final class EchoActor extends AbstractActor {
@Override
public Receive createReceive() {
return ReceiveBuilder.create()
.match(String.class, msg -> getSender().tell("echo:" + msg, getSelf()))
.build();
}
}
public static void main(String[] args) {
ActorSystem system = ActorSystem.create("demo");
ActorRef echo = system.actorOf(Props.create(EchoActor.class), "echo");
echo.tell("ping", ActorRef.noSender());
CompletionStage<Object> resp = Patterns.ask(echo, "ping", Duration.ofSeconds(3));
resp.toCompletableFuture().thenAccept(System.out::println).join();
system.terminate();
}
}
1.4 Demo 2:本机两个 ActorSystem 通过 remoting 通信(客户端/服务端)
这个 demo 展示了"服务端 ActorSystem 监听端口 + 客户端 ActorSystem 通过 actor path 访问"的基本形式;Flink RPC 的 URL 本质上也是在构造类似的远程 actor path(只不过路径固定挂在 /user/rpc/<endpointName> 下)。
java
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import org.apache.pekko.actor.AbstractActor;
import org.apache.pekko.actor.ActorRef;
import org.apache.pekko.actor.ActorSelection;
import org.apache.pekko.actor.ActorSystem;
import org.apache.pekko.actor.Props;
import org.apache.pekko.japi.pf.ReceiveBuilder;
import org.apache.pekko.pattern.Patterns;
import java.time.Duration;
public class RemotingDemo {
public static final class EchoActor extends AbstractActor {
@Override
public Receive createReceive() {
return ReceiveBuilder.create()
.match(String.class, msg -> getSender().tell("echo:" + msg, getSelf()))
.build();
}
}
private static Config remotingConfig(String hostname, int port) {
String conf =
"pekko.actor.provider = \"org.apache.pekko.remote.RemoteActorRefProvider\"\n"
+ "pekko.remote.artery.enabled = false\n"
+ "pekko.remote.classic.enabled-transports = [\"pekko.remote.classic.netty.tcp\"]\n"
+ "pekko.remote.classic.netty.tcp.hostname = \"" + hostname + "\"\n"
+ "pekko.remote.classic.netty.tcp.port = " + port + "\n";
return ConfigFactory.parseString(conf);
}
public static void main(String[] args) {
ActorSystem serverSystem = ActorSystem.create("srv", remotingConfig("127.0.0.1", 2551));
serverSystem.actorOf(Props.create(EchoActor.class), "echo");
ActorSystem clientSystem = ActorSystem.create("cli", remotingConfig("127.0.0.1", 0));
ActorSelection remoteEcho =
clientSystem.actorSelection("pekko://srv@127.0.0.1:2551/user/echo");
Patterns.ask(remoteEcho, "ping", Duration.ofSeconds(3))
.toCompletableFuture()
.thenAccept(System.out::println)
.join();
clientSystem.terminate();
serverSystem.terminate();
}
}
二、主题与核心组件职责
Flink 的 RPC 框架将"接口抽象"和"Pekko 实现"解耦:上层组件(如 JobManager、TaskManager、ResourceManager 等)面向接口编程,运行时由 Pekko 的实现类提供本地/远程通信能力。
- RpcSystem(接口) → PekkoRpcSystem(实现) :RPC 系统入口(
flink-rpc/flink-rpc-akka),负责创建本地/远程的RpcServiceBuilder,并生成/解析 RPC URL。 - RpcService(接口) → PekkoRpcService(实现) :RPC 服务本体(
flink-rpc/flink-rpc-akka),负责startServer/connect/getSelfGateway等能力,对外产出可调用的RpcGateway。 - RpcEndpoint(抽象基类) → 业务组件 Endpoint(继承形态) :端点基类(
flink-rpc/flink-rpc-core),由 JobManager/TaskManager/ResourceManager 等组件继承;构造时通过rpcService.startServer(this)注册服务端,提供主线程执行模型(runAsync/callAsync/getMainThreadExecutor)与生命周期钩子(onStart/onStop)。 - RpcGateway(接口) → JDK 动态代理(实现形态) :调用方拿到的"网关对象"通常不是某个固定实现类,而是
Proxy;它把方法调用委托给InvocationHandler。 - InvocationHandler(接口) → PekkoInvocationHandler(实现) :网关代理的核心实现(
flink-rpc/flink-rpc-akka),在invoke中把方法调用封装为RpcInvocation,通过 Pekko 的tell/ask发给 Actor 执行(本地或远程)。 - RpcUtils(工具类) :RPC 通用工具集(
flink-rpc/flink-rpc-core),用于抽取 endpoint 实现的RpcGateway接口集合、创建远程RpcService、以及提供停止/终止相关的通用能力。 - PekkoRpcServiceUtils(工具类) :Pekko RPC 适配工具集(
flink-rpc/flink-rpc-akka),负责RpcServiceBuilder组装与 RPC URL 规范(/user/rpc/<endpointName>)等细节。 - RobustActorSystem(实现类) → ActorSystemImpl(父类) :带可配置
UncaughtExceptionHandler的 ActorSystem(flink-rpc/flink-rpc-akka),用于在 Flink 中创建 Pekko ActorSystem,并在终止后过滤部分类加载异常噪音。 - SupervisorActor(Actor) :
/user/rpc根节点(flink-rpc/flink-rpc-akka),负责创建并监管每个 endpoint 对应的PekkoRpcActor。 - StartRpcActor(消息) :
SupervisorActor接收的启动消息(flink-rpc/flink-rpc-akka),包含endpointId与PropsFactory;由 supervisor 调用PropsFactory.create(terminationFuture)构造 endpoint 对应的PekkoRpcActor(或FencedPekkoRpcActor),并回传ActorRef + terminationFuture。 - PekkoRpcActor(Actor) :端点执行载体(
flink-rpc/flink-rpc-akka),接收RpcInvocation/RunAsync/CallAsync并在 Actor 线程内分发执行,同时承担返回值序列化与maximumFramesize校验的边界控制。
三、基础服务组件依赖
- Pekko (Actor 模型):底层的异步通信引擎,负责真正的网络传输、序列化与反序列化、消息队列排队及并发处理。Flink 通过 RPC 接口屏蔽了 Pekko 的具体实现细节。
- 动态代理 (InvocationHandler) :被用于拦截
RpcGateway接口的各种方法调用。通过代理模式,开发者可以像调用本地 Java 方法一样调用远程服务,而无需显式地打包消息和处理网络 IO。
四、启动入口与类图分析
相关核心模块与路径:
flink-rpc/flink-rpc-core模块:提供核心 RPC 服务接口定义。- 路径:
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/ - 核心类:
RpcService,RpcGateway等。
- 路径:
flink-rpc/flink-rpc-akka模块:基于 Pekko 实现的 RPC 系统。- 路径:
flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ - 核心类:
PekkoRpcSystem,PekkoInvocationHandler,RobustActorSystem,PekkoRpcActor等。
- 路径:
4.1 核心类继承图
implements
extends
implements
implements
implements
extends
extends
extends
extends
extends
implements
implements
implements
extends
extends
extends
receive/handle
build/config/url
create
ActorSystem
start /user/rpc
create endpoint actor
startServer/stopServer
tell/ask via ActorRef
extract gateways
remote service factory
builder/url utils
<<interface>>
RpcSystem
PekkoRpcSystem
<<interface>>
AutoCloseableAsync
<<interface>>
RpcService
PekkoRpcService
<<interface>>
RpcGateway
RpcEndpoint
FencedRpcEndpoint
Dispatcher
JobMaster
ResourceManager
TaskExecutor
<<interface>>
InvocationHandler
<<interface>>
PekkoBasedEndpoint
<<interface>>
RpcServer
PekkoInvocationHandler
<<utility>>
RpcUtils
<<utility>>
PekkoRpcServiceUtils
<<pekko>>
ActorSystemImpl
RobustActorSystem
<<pekko>>
AbstractActor
<<actor>>
SupervisorActor
<<message>>
StartRpcActor
<<actor>>
PekkoRpcActor
五、核心启动流程源码拆解
流程视角总结:
- 初始化通信底座(PekkoRpcService) :
PekkoRpcSystemcreate 出PekkoRpcService;而PekkoRpcService构造阶段会立刻startSupervisorActor(),把SupervisorActor挂到 ActorSystem 的/user/rpc根路径下。换句话说,"启动 RpcService"的关键动作,本质就是把这个 supervisor 拉起来作为后续 endpoint actor 的统一入口。 - 注册端点并启动 RpcActor(StartRpcActor → PekkoRpcActor) :业务组件继承
RpcEndpoint,在构造阶段调用rpcService.startServer(this)进入PekkoRpcService.startServer;内部会把 endpoint 的注册请求封装为StartRpcActor(通过SupervisorActor.startRpcActor(...)发给/user/rpc),由SupervisorActor创建并监管对应的PekkoRpcActor(或FencedPekkoRpcActor)。此时PekkoRpcActor持有RpcEndpoint实例,负责真正的消息接收、方法分发与返回值处理。 - 统一本地/远端调用(RpcGateway Proxy → PekkoInvocationHandler) :调用侧拿到的是
RpcGateway的动态代理对象,用来屏蔽"本地调用 vs 远端调用"的差异。所有方法调用都会进入PekkoInvocationHandler.invoke:它会先判断是否属于无需走 RPC 的基础方法(如地址/版本等)并本地处理;否则将调用封装成RpcInvocation,再通过 Pekko 的tell/ask投递到目标PekkoRpcActor执行,从而把本地/远端都收敛到同一套调用路径里。
5.1 RPC 通信调用流程图
RPC调用与消息分发
是
否
调用 RpcGateway 代理方法
PekkoInvocationHandler.invoke 拦截
是否为本地基础接口?
直接反射调用 method.invoke
invokeRpc 封装为 RpcInvocation
tell/ask 发送给底层 Actor
服务注册与代理生成
RpcService.startServer
注册 RpcEndpoint
生成 RpcGateway 动态代理
绑定 PekkoInvocationHandler
RPC系统初始化
PekkoRpcSystem 构建
初始化 RpcService
5.2 RpcService 创建与 URL 生成(RpcUtils / PekkoRpcServiceUtils)
这一部分关注"RpcService 是如何被创建出来的",以及"一个 RpcEndpoint 的 RPC URL 是如何拼出来的"。这两块分别由 flink-rpc-core 的通用工具类与 flink-rpc-akka 的 Pekko 适配工具类完成。
5.2.1 RpcUtils:抽取网关接口 + 快捷创建远程 RpcService
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java
java
public static Set<Class<? extends RpcGateway>> extractImplementedRpcGateways(Class<?> clazz) {
HashSet<Class<? extends RpcGateway>> interfaces = new HashSet<>();
while (clazz != null) {
for (Class<?> interfaze : clazz.getInterfaces()) {
if (RpcGateway.class.isAssignableFrom(interfaze)) {
interfaces.add((Class<? extends RpcGateway>) interfaze);
}
}
clazz = clazz.getSuperclass();
}
return interfaces;
}
public static RpcService createRemoteRpcService(
RpcSystem rpcSystem,
Configuration configuration,
@Nullable String externalAddress,
String externalPortRange,
@Nullable String bindAddress,
Optional<Integer> bindPort)
throws Exception {
RpcSystem.RpcServiceBuilder rpcServiceBuilder =
rpcSystem.remoteServiceBuilder(configuration, externalAddress, externalPortRange);
if (bindAddress != null) {
rpcServiceBuilder = rpcServiceBuilder.withBindAddress(bindAddress);
}
if (bindPort.isPresent()) {
rpcServiceBuilder = rpcServiceBuilder.withBindPort(bindPort.get());
}
return rpcServiceBuilder.createAndStart();
}
- extractImplementedRpcGateways :从一个实现类(通常是
RpcEndpoint/RpcGateway组合类)向上遍历父类链,把所有实现的RpcGateway接口收集出来,用于后续生成动态代理的接口列表。 - createRemoteRpcService :对
RpcSystem#remoteServiceBuilder的快捷封装,把外部地址、绑定地址、端口这些可选项"填齐",最终在createAndStart()时真正启动 RPC 系统。
5.2.2 PekkoRpcServiceUtils:RpcServiceBuilder + RPC URL 规范
flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceUtils.java
java
public static String getRpcUrl(
String hostname,
int port,
String endpointName,
AddressResolution addressResolution,
Configuration config)
throws UnknownHostException {
final boolean sslEnabled =
config.get(RpcOptions.SSL_ENABLED) && SecurityOptions.isInternalSSLEnabled(config);
return getRpcUrl(
hostname,
port,
endpointName,
addressResolution,
sslEnabled ? Protocol.SSL_TCP : Protocol.TCP);
}
private static String internalRpcUrl(
String endpointName, Optional<RemoteAddressInformation> remoteAddressInformation) {
final String protocolPrefix =
remoteAddressInformation
.map(rai -> protocolToString(rai.getProtocol()))
.orElse("pekko");
final Optional<String> optionalHostnameAndPort =
remoteAddressInformation.map(RemoteAddressInformation::getHostnameAndPort);
final StringBuilder url = new StringBuilder(String.format("%s://flink", protocolPrefix));
optionalHostnameAndPort.ifPresent(hostPort -> url.append("@").append(hostPort));
url.append("/user/").append(SUPERVISOR_NAME).append("/").append(endpointName);
return url.toString();
}
static PekkoRpcServiceBuilder remoteServiceBuilder(
Configuration configuration, @Nullable String externalAddress, String externalPortRange) {
return new PekkoRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange);
}
- getRpcUrl / internalRpcUrl :统一了 Flink RPC URL 的格式:
<protocol>://flink[@host:port]/user/rpc/<endpointName>;其中SUPERVISOR_NAME固定为rpc,对应 ActorSystem 的/user/rpc根节点。 - remoteServiceBuilder/localServiceBuilder :把"如何启动 ActorSystem、如何配置线程池/最大帧大小/SSL"等细节封装进 builder,最终会产出
PekkoRpcService。
5.2.3 PekkoRpcServiceBuilder#createAndStart:启动 ActorSystem 并构造 PekkoRpcService
同样位于 PekkoRpcServiceUtils 内部的 builder:
java
public PekkoRpcService createAndStart() throws Exception {
return createAndStart(PekkoRpcService::new);
}
public PekkoRpcService createAndStart(
TriFunction<ActorSystem, PekkoRpcServiceConfiguration, ClassLoader, PekkoRpcService>
constructor)
throws Exception {
if (actorSystemExecutorConfiguration == null) {
actorSystemExecutorConfiguration =
PekkoUtils.getForkJoinExecutorConfig(
ActorSystemBootstrapTools.getForkJoinExecutorConfiguration(configuration));
}
final ActorSystem actorSystem;
try (TemporaryClassLoaderContext ignored =
TemporaryClassLoaderContext.of(getClass().getClassLoader())) {
if (externalAddress == null) {
actorSystem =
ActorSystemBootstrapTools.startLocalActorSystem(
configuration,
actorSystemName,
logger,
actorSystemExecutorConfiguration,
customConfig);
} else {
actorSystem =
ActorSystemBootstrapTools.startRemoteActorSystem(
configuration,
actorSystemName,
externalAddress,
externalPortRange,
bindAddress,
Optional.ofNullable(bindPort),
logger,
actorSystemExecutorConfiguration,
customConfig);
}
}
return constructor.apply(
actorSystem,
PekkoRpcServiceConfiguration.fromConfiguration(configuration),
RpcService.class.getClassLoader());
}
- 关键点 :真正的"RPC 系统启动"发生在这里:先选择本地/远程模式启动
ActorSystem,再用构造器(默认PekkoRpcService::new)把ActorSystem包装成RpcService语义。
5.3 RobustActorSystem:可配置 UncaughtExceptionHandler 的 ActorSystemImpl
flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystem.java
RobustActorSystem 是对 Pekko ActorSystemImpl 的轻量增强:允许注入 Thread.UncaughtExceptionHandler,并在 ActorSystem 终止后过滤一类"类加载异常噪音"(常见于 RPC classloader 已关闭但 Pekko 的 shutdown 过程仍有异步清理动作)。
java
public abstract class RobustActorSystem extends ActorSystemImpl {
public static RobustActorSystem create(String name, Config applicationConfig) {
return create(name, applicationConfig, FatalExitExceptionHandler.INSTANCE);
}
static RobustActorSystem create(
String name,
Config applicationConfig,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler) {
return create(
name,
ActorSystemSetup.create(
BootstrapSetup.create(
Optional.empty(),
Optional.of(applicationConfig),
Optional.empty())),
uncaughtExceptionHandler);
}
private static RobustActorSystem create(
String name,
ActorSystemSetup setup,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler) {
...
final PostShutdownClassLoadingErrorFilter filter =
new PostShutdownClassLoadingErrorFilter(uncaughtExceptionHandler);
final RobustActorSystem robustActorSystem =
new RobustActorSystem(name, appConfig, classLoader, defaultEC, setup) {
@Override
public Thread.UncaughtExceptionHandler uncaughtExceptionHandler() {
return filter;
}
};
robustActorSystem.registerOnTermination(filter::notifyShutdownComplete);
robustActorSystem.start();
return robustActorSystem;
}
}
它在 Flink 的 Pekko 工具类中被使用,用于创建 ActorSystem:
java
public static ActorSystem createActorSystem(String actorSystemName, Config config) {
InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
return RobustActorSystem.create(actorSystemName, config);
}
- 异常处理可控 :把默认的"遇到未捕获异常直接退出"策略(
FatalExitExceptionHandler)显式化,并允许在测试或特殊场景下注入其他 handler。 - 终止后的类加载异常过滤 :ActorSystem 已终止后,忽略
NoClassDefFoundError/ClassNotFoundException这类由 shutdown 尾部动作触发的异常,减少误报与日志噪音。
5.4 SupervisorActor:/user/rpc 根节点与 RpcActor 的创建
flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/SupervisorActor.java
java
public static String getActorName() {
return PekkoRpcServiceUtils.SUPERVISOR_NAME;
}
public static ActorRef startSupervisorActor(
ActorSystem actorSystem, Executor terminationFutureExecutor) {
final Props supervisorProps =
Props.create(SupervisorActor.class, terminationFutureExecutor)
.withDispatcher("pekko.actor.supervisor-dispatcher");
return actorSystem.actorOf(supervisorProps, getActorName());
}
public static StartRpcActorResponse startRpcActor(
ActorRef supervisor, StartRpcActor.PropsFactory propsFactory, String endpointId) {
return Patterns.ask(
supervisor,
createStartRpcActorMessage(propsFactory, endpointId),
RpcUtils.INF_DURATION)
.toCompletableFuture()
.thenApply(StartRpcActorResponse.class::cast)
.join();
}
StartRpcActor 是 SupervisorActor 收到的"启动某个 endpoint 对应 RpcActor"的消息载体,核心就是两个字段:endpointId(Actor 名称/路径的一部分)与 PropsFactory(如何构造 Props,也就是如何构造 PekkoRpcActor)。
java
static final class StartRpcActor {
private final PropsFactory propsFactory;
private final String endpointId;
private StartRpcActor(PropsFactory propsFactory, String endpointId) {
this.propsFactory = propsFactory;
this.endpointId = endpointId;
}
public String getEndpointId() {
return endpointId;
}
public PropsFactory getPropsFactory() {
return propsFactory;
}
private static StartRpcActor create(PropsFactory propsFactory, String endpointId) {
return new StartRpcActor(propsFactory, endpointId);
}
interface PropsFactory {
Props create(CompletableFuture<Void> terminationFuture);
}
}
- SupervisorActor 的定位 :ActorSystem 的
/user/rpc根节点(由SUPERVISOR_NAME = "rpc"决定),负责创建并监管每一个PekkoRpcActor(也就是每个RpcEndpoint对应的 Actor)。 - startRpcActor 的交互模式 :通过
Patterns.ask向 supervisor 发送StartRpcActor消息;成功时返回新建的ActorRef以及 terminationFuture,失败时返回异常包装。 - endpointId 的含义 :在
SupervisorActor里会用getContext().actorOf(rpcActorProps, endpointId)创建子 actor,endpointId直接成为该PekkoRpcActor的 actor name,因此它会体现在 actor path 中(/user/rpc/<endpointId>)。 - PropsFactory 的含义 :让"如何构造
PekkoRpcActor"延迟到 supervisor 内部执行,并把terminationFuture注入进去,便于把 actor 的终止状态向外转发(最终返回到ActorRegistration里)。 - INF_DURATION 的来源 :这里使用了
RpcUtils.INF_DURATION作为启动阶段的 ask 超时上限,避免启动慢导致误判超时(上限由 Pekko 自身调度器最大延迟约束)。
5.5 PekkoRpcActor:端点的 Actor 执行载体(消息分发 + 序列化边界)
flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActor.java
PekkoRpcActor 是承载 RpcEndpoint 的 Pekko Actor:它接收 RpcInvocation、RunAsync、CallAsync 等消息,并在 Actor 线程里调用 endpoint 的方法(同时通过状态机控制"是否允许处理消息")。
java
class PekkoRpcActor<T extends RpcEndpoint & RpcGateway> extends AbstractActor {
protected final T rpcEndpoint;
@Nonnull private State state;
@Override
public Receive createReceive() {
return ReceiveBuilder.create()
.match(RemoteHandshakeMessage.class, this::handleHandshakeMessage)
.match(ControlMessages.class, this::handleControlMessage)
.matchAny(this::handleMessage)
.build();
}
private void handleMessage(final Object message) {
if (state.isRunning()) {
mainThreadValidator.enterMainThread();
try {
handleRpcMessage(message);
} finally {
mainThreadValidator.exitMainThread();
}
} else {
sendErrorIfSender(new EndpointNotStartedException(...));
}
}
}
java
private void handleRpcInvocation(RpcInvocation rpcInvocation) {
...
if (rpcMethod.getReturnType().equals(Void.TYPE)) {
runWithContextClassLoader(
() -> capturedRpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()),
flinkClassLoader);
} else {
final Object result = runWithContextClassLoader(..., flinkClassLoader);
final boolean isLocalRpcInvocation = rpcMethod.getAnnotation(Local.class) != null;
if (result instanceof CompletableFuture) {
sendAsyncResponse((CompletableFuture<?>) result, methodName, isLocalRpcInvocation);
} else {
sendSyncResponse(result, methodName, isLocalRpcInvocation);
}
}
}
- 消息入口统一 :
createReceive只关心三类入口:握手(版本/网关兼容性)、控制消息(START/STOP/TERMINATE)、业务消息(RunAsync/CallAsync/RpcInvocation)。 - 状态机保护 :未 START 的阶段会丢弃消息并返回
EndpointNotStartedException;START 后才进入"可处理消息"的运行态。 - 主线程语义对齐 :通过
MainThreadValidatorUtil标记"当前正处在 endpoint 主线程上下文";配合RpcEndpoint#validateRunsInMainThread在测试期开启断言时做并发误用检测。 - Invocation 执行模型 :
RpcInvocation通过反射定位 endpoint 方法并调用;void走 fire-and-forget;非void则区分同步返回与CompletableFuture异步返回。 - 序列化边界 :对远端 sender(或
forceSerialization且非@Local)会把结果封装为RpcSerializedValue,并校验maximumFramesize,避免返回值过大撑爆 RPC 传输上限。
5.6 RpcEndpoint:端点基类、主线程模型与生命周期
flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
RpcEndpoint 是 Flink RPC 体系里"业务组件侧"的基类:组件继承它获得统一的生命周期(start/stop/close)、主线程模型(main thread executor)以及对外暴露网关(gateway)的基础能力。
java
public abstract class RpcEndpoint implements RpcGateway, AutoCloseableAsync {
private final RpcService rpcService;
private final String endpointId;
protected final RpcServer rpcServer;
private final MainThreadExecutor mainThreadExecutor;
private boolean isRunning;
protected RpcEndpoint(final RpcService rpcService, final String endpointId) {
this.rpcService = checkNotNull(rpcService, "rpcService");
this.endpointId = checkNotNull(endpointId, "endpointId");
this.rpcServer = rpcService.startServer(this);
this.mainThreadExecutor =
new MainThreadExecutor(rpcServer, this::validateRunsInMainThread, endpointId);
registerResource(this.mainThreadExecutor);
}
public final void start() {
rpcServer.start();
}
@Override
public final CompletableFuture<Void> closeAsync() {
rpcService.stopServer(rpcServer);
return getTerminationFuture();
}
public <C extends RpcGateway> C getSelfGateway(Class<C> selfGatewayType) {
return rpcService.getSelfGateway(selfGatewayType, rpcServer);
}
public CompletableFuture<Void> getTerminationFuture() {
return rpcServer.getTerminationFuture();
}
protected void runAsync(Runnable runnable) {
rpcServer.runAsync(runnable);
}
protected <V> CompletableFuture<V> callAsync(Callable<V> callable, Duration timeout) {
return rpcServer.callAsync(callable, timeout);
}
}
- RpcEndpoint 与 RpcServer :构造函数里直接
rpcService.startServer(this),把"当前 endpoint"注册为可接收 RPC 的服务端,并返回可控的RpcServer(底层就是代理 + ActorRef 的封装)。 - 生命周期分层 :对用户暴露
start()/closeAsync();内部由 RpcService 在合适时机回调internalCallOnStart()/internalCallOnStop()来切换运行态并触发onStart()/onStop()钩子。 - 单线程模型 :通过
MainThreadExecutor让"改状态的逻辑"串行执行;validateRunsInMainThread()在开启断言时用于主线程校验,帮助在测试期暴露并发误用。 - 网关自举(self gateway) :
getSelfGateway()让 endpoint 在本进程内也通过"网关接口 + 代理"访问自己,保证调用路径与远端一致(统一走网关语义)。
5.7 RpcService 通信底座
该接口定义了连接远程服务、启动本地服务以及获取执行线程池等基础能力。
java
public interface RpcService extends AutoCloseableAsync {
// 获取自身的代理网关
<C extends RpcGateway> C getSelfGateway(Class<C> selfGatewayType, RpcServer rpcServer);
// 连接到远程的 RPC Server,返回用于通信的网关代理
<C extends RpcGateway> CompletableFuture<C> connect(String address, Class<C> clazz);
// 启动一个 RPC Server 包装传入的 rpcEndpoint
<C extends RpcEndpoint & RpcGateway> RpcServer startServer(C rpcEndpoint);
// 获取用于定时任务的 ScheduledExecutor
ScheduledExecutor getScheduledExecutor();
}
- 核心职责 :为
RpcEndpoint提供生命周期管理(启动/停止),以及获取通信代理对象(connect获取远端代理,getSelfGateway获取本地代理)。 startServer方法:将具体的端点实现暴露出去,使得它可以被其他节点连接和调用。
5.8 PekkoInvocationHandler 拦截代理请求
作为动态代理的核心,PekkoInvocationHandler 负责拦截对 RpcGateway 的所有方法调用,并根据方法的归属类决定是本地执行还是转发给 Actor。
java
@Override
public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
Class<?> declaringClass = method.getDeclaringClass();
Object result;
// 如果调用的方法属于以下基础框架接口,则直接在当前 handler 对象上进行本地反射调用
if (declaringClass.equals(PekkoBasedEndpoint.class)
|| declaringClass.equals(Object.class)
|| declaringClass.equals(RpcGateway.class)
|| declaringClass.equals(StartStoppable.class)
|| declaringClass.equals(MainThreadExecutable.class)
|| declaringClass.equals(RpcServer.class)) {
result = method.invoke(this, args);
} else if (declaringClass.equals(FencedRpcGateway.class)) {
// 异常处理逻辑...
throw new UnsupportedOperationException("...");
} else {
// 业务方法调用,进行 RPC 封装并发送
result = invokeRpc(method, args);
}
return result;
}
- 方法拦截 :使用 JDK 动态代理的
invoke方法,所有对网关方法的调用都会进入这里。 - 本地直接调用 :针对框架自身的接口调用(如获取网关地址
getAddress),直接本地处理,无需走 RPC 网络链路。 - 业务方法 RPC 化 :如果是业务层面的 RPC 方法,则走
invokeRpc方法将其转换成 Actor 消息。
5.9 将方法调用封装为 Actor 消息 (invokeRpc)
在 invokeRpc 中,方法调用被包装为可序列化的 RpcInvocation 对象,并发送给对应的 Pekko Actor 处理。
java
private Object invokeRpc(Method method, Object[] args) throws Exception {
String methodName = method.getName();
Class<?>[] parameterTypes = method.getParameterTypes();
final boolean isLocalRpcInvocation = method.getAnnotation(Local.class) != null;
// 1. 创建 RPC 调用消息,封装了类名、方法名、参数类型及实际参数
final RpcInvocation rpcInvocation = createRpcInvocationMessage(
method.getDeclaringClass().getSimpleName(),
methodName, isLocalRpcInvocation, parameterTypes, args);
Class<?> returnType = method.getReturnType();
final Object result;
// 2. 根据方法的返回类型决定通信模式
if (Objects.equals(returnType, Void.TYPE)) {
// 无返回值:使用 tell 进行单向发送(Fire and Forget)
tell(rpcInvocation);
result = null;
} else {
// 有返回值:使用 ask 模式进行双向通信,等待 Future 结果
final CompletableFuture<?> resultFuture = ask(rpcInvocation, futureTimeout)
.thenApply(resultValue -> deserializeValueIfNeeded(resultValue, method, flinkClassLoader));
final CompletableFuture<Object> completableFuture = new CompletableFuture<>();
// 处理异步结果与超时...
// 如果返回值声明就是 CompletableFuture,直接返回
if (Objects.equals(returnType, CompletableFuture.class)) {
result = completableFuture;
} else {
// 如果是同步阻塞返回值,则阻塞等待
result = completableFuture.get(futureTimeout.toMillis(), TimeUnit.MILLISECONDS);
}
}
return result;
}
- 创建 RpcInvocation :将方法名、参数信息封装成消息。如果检测到是本地调用,则封装为
LocalRpcInvocation,以省去不必要的序列化开销;否则封装为RemoteRpcInvocation。 - 单向发送 (
tell) :如果方法返回值为void,底层直接使用 Pekko 的tell,发送消息后立即返回,不阻塞。 - 异步响应 (
ask) :如果方法有返回值(尤其是CompletableFuture),则使用 Pekko 的ask模式,并将其结果转换为 Java 的CompletableFuture返回给调用方,从而实现了全异步的非阻塞 RPC 通信。
六、总结
- 本文完整梳理了 Flink 基于 Pekko(Akka) 的 RPC 通信体系,重点聚焦
PekkoRpcService、SupervisorActor/StartRpcActor、PekkoRpcActor、RpcGateway Proxy与PekkoInvocationHandler这些关键组件在"注册端点、生成网关、消息投递与执行"的职责边界。 - 由于本文以"组件机制拆解"为主,并未把它们放进具体的 Flink 进程/组件启动链路中串起来;后续会在
ResourceManager、Dispatcher、MetricService等初始化与运行流程里,把这些 RPC 机制嵌入到实际启动时序中进行串联。