Flink 源码之RPC调用

Flink源码分析系列文档目录

请点击:Flink 源码分析系列文档目录

简介

本篇为大家分析Flink底层RPC调用的执行过程和原理。Flink RPC使用了Akka Actor框架。建议阅读前需要Akka Actor的基础知识。

Flink为了简化Actor的使用,对Actor做了一系列封装。定义了如下几个重要的接口或类:

  • RpcService:定义了RPC服务端(调用和被调用端)的一系列行为,远程调用需要指定地址和端口号。包含服务的启停,和连接其他远程的RpcService。
  • AkkaRpcService:RpcService的实现类。
  • RpcServer:RPC服务,它自己也是个RpcGateway
  • AkkaInvocationHandler:RpcServer的实现类。RpcService连接其他远程RpcService返回的实际上是一个代理类。这个代理类的真实执行逻辑位于AkkaInvocationHandler
  • RpcGateway:RPC网关,调用其他RpcService的入口。一个RpcService连接其他远程RpcService返回的对象是RpcGateway类型。通常RpcGateway还实现了其他接口,这个接口和远端被调用的RpcService实现的接口相同。上面说过,RpcService连接其他远程RpcService返回的实际上是一个代理类,我们调用这个代理类的时候,底层会通过Akka调用远程RpcService的同名方法。
  • RpcEndpoint:RPC被调用端,需要持有一个RpcService,并且实现自己的业务逻辑接口,以供RpcGateway远程调用的时候执行。
Flink RPC

Flink封装后的Akka使用起来非常简单。我们可以参考Flink单元测试RemoteAkkaRpcActorTestcanRespondWithSerializedValueRemotely方法,它通过remoteGateway远程调用AkkaRpcActorTest.SerializedValueRespondingEndpointgetSerializedValueSynchronously方法。代码如下所示:

@Test
public void canRespondWithSerializedValueRemotely() throws Exception {
    try (final AkkaRpcActorTest.SerializedValueRespondingEndpoint endpoint =
            new AkkaRpcActorTest.SerializedValueRespondingEndpoint(rpcService)) {
        endpoint.start();

        final AkkaRpcActorTest.SerializedValueRespondingGateway remoteGateway =
                otherRpcService
                        .connect(
                                endpoint.getAddress(),
                                AkkaRpcActorTest.SerializedValueRespondingGateway.class)
                        .join();

        assertThat(
                remoteGateway.getSerializedValueSynchronously(),
                equalTo(AkkaRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE));

        final CompletableFuture> responseFuture =
                remoteGateway.getSerializedValue();

        assertThat(
                responseFuture.get(),
                equalTo(AkkaRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE));
    }
}

单元测试中如何创建RpcService的方法没有贴出。

本篇从TaskManagerRunner创建RpcService开始,分析Flink封装Akka Actor的方法和整个调用流程。

RpcService

TaskManager创建RpcService

TaskManager的createRpcService方法根据Flink的配置文件和高可用服务,创建出RpcService。

TaskManagerRunnercreateRpcService方法如下:

@VisibleForTesting
static RpcService createRpcService(
        final Configuration configuration, final HighAvailabilityServices haServices)
        throws Exception {

    checkNotNull(configuration);
    checkNotNull(haServices);

    return AkkaRpcServiceUtils.createRemoteRpcService(
            configuration,
            determineTaskManagerBindAddress(configuration, haServices),
            configuration.getString(TaskManagerOptions.RPC_PORT),
            configuration.getString(TaskManagerOptions.BIND_HOST),
            configuration.getOptional(TaskManagerOptions.RPC_BIND_PORT));
}

创建RpcService的逻辑在AkkaRpcServiceUtils工具类中。接下来我们谈一下RpcService和它的创建过程。

RpcService定义

负责使用Akka Actor执行RPC。拥有一个子类AkkaRpcService。一个进程拥有一个RpcService。用于统筹RPC调用服务,连接到本地Endpoint启动服务,供远端调用(RpcEndpoint是被调用端),或者连接到远程RPC服务,创建出一个RpcGateway(调用端),从而可以调用远端。

RpcService接口代码如下所示:

public interface RpcService {

    /**
     * Return the hostname or host address under which the rpc service can be reached. If the rpc
     * service cannot be contacted remotely, then it will return an empty string.
     *
     * @return Address of the rpc service or empty string if local rpc service
     */
    // 获取RPC服务的地址
    // 如果是本地RPC服务的话,返回空
    String getAddress();

    /**
     * Return the port under which the rpc service is reachable. If the rpc service cannot be
     * contacted remotely, then it will return -1.
     *
     * @return Port of the rpc service or -1 if local rpc service
     */
    // 返回RPC服务端口号
    // 如果是本地RPC服务,返回-1
    int getPort();

    /**
     * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can be
     * used to communicate with the rpc server. If the connection failed, then the returned future
     * is failed with a {@link RpcConnectionException}.
     *
     * @param address Address of the remote rpc server
     * @param clazz Class of the rpc gateway to return
     * @param  Type of the rpc gateway to return
     * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the
     *     connection attempt failed
     */
    // 根据提供的地址,连接到远程RPC服务
    // 返回C类型RPC网关,用于和远端通信
     CompletableFuture connect(String address, Class clazz);

    /**
     * Connect to a remote fenced rpc server under the provided address. Returns a fenced rpc
     * gateway which can be used to communicate with the rpc server. If the connection failed, then
     * the returned future is failed with a {@link RpcConnectionException}.
     *
     * @param address Address of the remote rpc server
     * @param fencingToken Fencing token to be used when communicating with the server
     * @param clazz Class of the rpc gateway to return
     * @param  Type of the fencing token
     * @param  Type of the rpc gateway to return
     * @return Future containing the fenced rpc gateway or an {@link RpcConnectionException} if the
     *     connection attempt failed
     */
    // 创建一个具有Fence功能的RPC网关
    // Fence是防止脑裂的机制,我们在AkkaRpcActor分析Fencing机制原理
    > CompletableFuture connect(
            String address, F fencingToken, Class clazz);

    /**
     * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint.
     *
     * @param rpcEndpoint Rpc protocol to dispatch the rpcs to
     * @param  Type of the rpc endpoint
     * @return Self gateway to dispatch remote procedure calls to oneself
     */
    // 启动RPC服务,将接收到的远程请求发送给rpcEndpoint处理
     RpcServer startServer(C rpcEndpoint);

    /**
     * Fence the given RpcServer with the given fencing token.
     *
     * 

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 {@link FencedRpcEndpoint}. * * @param rpcServer to fence with the given fencing token * @param fencingToken to fence the RpcServer with * @param type of the fencing token * @return Fenced RpcServer */ // 和上面的方法一样,只不过启用了防脑裂功能 RpcServer fenceRpcServer(RpcServer rpcServer, F fencingToken); /** * Stop the underlying rpc server of the provided self gateway. * * @param selfGateway Self gateway describing the underlying rpc server */ // 停止RPC服务 void stopServer(RpcServer selfGateway); /** * Trigger the asynchronous stopping of the {@link RpcService}. * * @return Future which is completed once the {@link RpcService} has been fully stopped. */ // 异步停止RPC服务 CompletableFuture stopService(); /** * Returns a future indicating when the RPC service has been shut down. * * @return Termination future */ // 返回一个CompletableFuture,在RPC服务完全关闭之后调用 CompletableFuture getTerminationFuture(); /** * Gets the executor, provided by this RPC service. This executor can be used for example for * the {@code handleAsync(...)} or {@code 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 {@link RpcEndpoint}. For such operations, one needs to use the {@link * RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that {@code RpcEndpoint}. * * @return The execution context provided by the RPC service */ // 获取RPC服务执行线程,可用于handleAsync等异步逻辑执行 Executor getExecutor(); /** * Gets a scheduled executor from the RPC service. This executor can be used to schedule tasks * to be executed in the future. * *

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 {@link RpcEndpoint}. For such operations, one needs to use the {@link * RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that {@code RpcEndpoint}. * * @return The RPC service provided scheduled executor */ // 获取定时任务线程池 ScheduledExecutor getScheduledExecutor(); /** * Execute the runnable in the execution context of this RPC Service, as returned by {@link * #getExecutor()}, after a scheduled delay. * * @param runnable Runnable to be executed * @param delay The delay after which the runnable will be executed */ // 设置一个定时任务,在ScheduledExecutor中执行 ScheduledFuture scheduleRunnable(Runnable runnable, long delay, TimeUnit unit); /** * Execute the given runnable in the executor of the RPC service. This method can be used to run * code outside of the main thread of a {@link 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 {@link RpcEndpoint}. For such operations, one needs to use the {@link * RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that {@code RpcEndpoint}. * * @param runnable to execute */ // 在RPC服务线程池中运行runnable void execute(Runnable runnable); /** * Execute the given callable and return its result as a {@link CompletableFuture}. This method * can be used to run code outside of the main thread of a {@link 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 {@link RpcEndpoint}. For such operations, one needs to use the {@link * RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that {@code RpcEndpoint}. * * @param callable to execute * @param is the return value type * @return Future containing the callable's future result */ // 在RPC服务线程池异步运行callable任务,异步结果以CompletableFuture形式返回 CompletableFuture execute(Callable callable); }

AkkaRpcServiceUtils

一个负责创建AkkaRpcService的工具类。

我们继续第一节的创建RpcService的过程。查看AkkaRpcServiceUtilscreateRemoteRpcService方法,如下所示:

public static AkkaRpcService createRemoteRpcService(
        Configuration configuration,
        @Nullable String externalAddress,
        String externalPortRange,
        @Nullable String bindAddress,
        @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort)
        throws Exception {
    // 创建一个serviceBuilder
    final AkkaRpcServiceBuilder akkaRpcServiceBuilder =
            AkkaRpcServiceUtils.remoteServiceBuilder(
                    configuration, externalAddress, externalPortRange);

    // 传入bind地址和bind端口号配置
    if (bindAddress != null) {
        akkaRpcServiceBuilder.withBindAddress(bindAddress);
    }

    bindPort.ifPresent(akkaRpcServiceBuilder::withBindPort);

    // 创建并启动RpcService
    return akkaRpcServiceBuilder.createAndStart();
}

AkkaRpcService通过建造者模式构建,在给予akkaRpcServiceBuilder足够配置信息后,调用createAndStart方法创建出AkkaRpcService

public AkkaRpcService createAndStart() throws Exception {
    // 获取线程池并行度配置
    if (actorSystemExecutorConfiguration == null) {
        actorSystemExecutorConfiguration =
                BootstrapTools.ForkJoinExecutorConfiguration.fromConfiguration(
                        configuration);
    }

    final ActorSystem actorSystem;

    // 如果没有配置外部访问地址,创建本地ActorSystem
    if (externalAddress == null) {
        // create local actor system
        actorSystem =
                BootstrapTools.startLocalActorSystem(
                        configuration,
                        actorSystemName,
                        logger,
                        actorSystemExecutorConfiguration,
                        customConfig);
    } else {
        // 否则创建一个远程ActorSystem
        // create remote actor system
        actorSystem =
                BootstrapTools.startRemoteActorSystem(
                        configuration,
                        actorSystemName,
                        externalAddress,
                        externalPortRange,
                        bindAddress,
                        Optional.ofNullable(bindPort),
                        logger,
                        actorSystemExecutorConfiguration,
                        customConfig);
    }

    // 返回AkkaRpcService实例
    // 在后面章节分析
    return new AkkaRpcService(
            actorSystem, AkkaRpcServiceConfiguration.fromConfiguration(configuration));
}

紧接着我们分析ActorSystem的创建过程。

BootstrapTools

startLocalActorSystem

此方法创建一个用于本地调用的ActorSystem

BootstrapToolsstartLocalActorSystem方法内容如下:

public static ActorSystem startLocalActorSystem(
        Configuration configuration,
        String actorSystemName,
        Logger logger,
        ActorSystemExecutorConfiguration actorSystemExecutorConfiguration,
        Config customConfig)
        throws Exception {

    logger.info("Trying to start local actor system");

    try {
        // 获取Akka配置,externalAddress和bindAddress为空
        // 对应的是一个本地的ActorSystem配置
        Config akkaConfig =
                AkkaUtils.getAkkaConfig(
                        configuration,
                        scala.Option.empty(),
                        scala.Option.empty(),
                        actorSystemExecutorConfiguration.getAkkaConfig());

        // 如果有自定义配置,将基本配置和自定义配置拼装起来,重复的配置项基本配置优先
        if (customConfig != null) {
            akkaConfig = customConfig.withFallback(akkaConfig);
        }

        // 启动ActorSystem
        return startActorSystem(akkaConfig, actorSystemName, logger);
    } catch (Throwable t) {
        throw new Exception("Could not create actor system", t);
    }
}

startRemoteActorSystem

创建一个可以远程调用的ActorSystem

startRemoteActorSystem方法内容如下:

public static ActorSystem startRemoteActorSystem(
        Configuration configuration,
        String actorSystemName,
        String externalAddress,
        String externalPortRange,
        String bindAddress,
        @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort,
        Logger logger,
        ActorSystemExecutorConfiguration actorSystemExecutorConfiguration,
        Config customConfig)
        throws Exception {

    // parse port range definition and create port iterator
    Iterator portsIterator;
    try {
        // 解析端口范围字符串,生成整数类型端口集合
        portsIterator = NetUtils.getPortRangeFromString(externalPortRange);
    } catch (Exception e) {
        throw new IllegalArgumentException(
                "Invalid port range definition: " + externalPortRange);
    }

    while (portsIterator.hasNext()) {
        final int externalPort = portsIterator.next();

        // 逐个尝试范围内的端口,直到启动ActorSystem成功
        try {
            return startRemoteActorSystem(
                    configuration,
                    actorSystemName,
                    externalAddress,
                    externalPort,
                    bindAddress,
                    bindPort.orElse(externalPort),
                    logger,
                    actorSystemExecutorConfiguration,
                    customConfig);
        } catch (Exception e) {
            // we can continue to try if this contains a netty channel exception
            Throwable cause = e.getCause();
            if (!(cause instanceof org.jboss.netty.channel.ChannelException
                    || cause instanceof java.net.BindException)) {
                throw e;
            } // else fall through the loop and try the next port
        }
    }

    // 如果执行到这一步,说明范围内所有端口都无法使用
    // if we come here, we have exhausted the port range
    throw new BindException(
            "Could not start actor system on any port in port range " + externalPortRange);
}

注:
Flink1.11之后支持TaskManager JobManager本地和远程使用不同的地址和端口,从而支持Docker和NAT端口映射。具体内容参见Flink-15911和Flink-15154。

配置远程的监听地址和端口:

  • jobmanager.rpc.address
  • jobmanager.rpc.port
  • taskmanager.host
  • taskmanager.rpc.port
  • taskmanager.data.port

配置本地的监听地址和端口:

  • jobmanager.bind-host
  • jobmanager.rpc.bind-port
  • taskmanager.bind-host
  • taskmanager.rpc.bind-port
  • taskmanager.data.bind-port

上面的方法最后调用了重载方法。该方法内容和startLocalActorSystem类似。

private static ActorSystem startRemoteActorSystem(
    Configuration configuration,
    String actorSystemName,
    String externalAddress,
    int externalPort,
    String bindAddress,
    int bindPort,
    Logger logger,
    ActorSystemExecutorConfiguration actorSystemExecutorConfiguration,
    Config customConfig)
    throws Exception {

    // 将地址和端口规范化后返回
    String externalHostPortUrl =
        NetUtils.unresolvedHostAndPortToNormalizedString(externalAddress, externalPort);
    String bindHostPortUrl =
        NetUtils.unresolvedHostAndPortToNormalizedString(bindAddress, bindPort);
    logger.info(
        "Trying to start actor system, external address {}, bind address {}.",
        externalHostPortUrl,
        bindHostPortUrl);

    try {
        // 和startLocalActorSystem一样
        // 多了传入externalAddress port,和bindAddress以及port
        Config akkaConfig =
            AkkaUtils.getAkkaConfig(
            configuration,
            new Some<>(new Tuple2<>(externalAddress, externalPort)),
            new Some<>(new Tuple2<>(bindAddress, bindPort)),
            actorSystemExecutorConfiguration.getAkkaConfig());

        if (customConfig != null) {
            akkaConfig = customConfig.withFallback(akkaConfig);
        }

        return startActorSystem(akkaConfig, actorSystemName, logger);
    } catch (Throwable t) {
        if (t instanceof ChannelException) {
            Throwable cause = t.getCause();
            if (cause != null && t.getCause() instanceof BindException) {
                throw new IOException(
                    "Unable to create ActorSystem at address "
                    + bindHostPortUrl
                    + " : "
                    + cause.getMessage(),
                    t);
            }
        }
        throw new Exception("Could not create actor system", t);
    }
}

startActorSystem

最后,通过AkkaUtils创建出ActorSystem

private static ActorSystem startActorSystem(
    Config akkaConfig, String actorSystemName, Logger logger) {
    logger.debug("Using akka configuration\n {}", akkaConfig);
    ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig);

    logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem));
    return actorSystem;
}

AkkaUtils

AkkaUtils负责生成Akka配置和创建ActorSystem。这里我们主要关注它生成Akka配置的方法。

下面是getAkkaConfig方法,注意该类使用Scala编写。

@throws(classOf[UnknownHostException])
def getAkkaConfig(configuration: Configuration,
                  externalAddress: Option[(String, Int)],
                  bindAddress: Option[(String, Int)],
                  executorConfig: Config): Config = {
    // 获取Akka基本配置,以及executor配置
    val defaultConfig = getBasicAkkaConfig(configuration).withFallback(executorConfig)

    // 根据bindHostname,bindPort,externalHostname,externalPort是否传入,生成不同的配置
    externalAddress match {

        case Some((externalHostname, externalPort)) =>

        bindAddress match {

            case Some((bindHostname, bindPort)) =>

            val remoteConfig = getRemoteAkkaConfig(
                configuration, bindHostname, bindPort, externalHostname, externalPort)

            remoteConfig.withFallback(defaultConfig)

            case None =>
            val remoteConfig = getRemoteAkkaConfig(configuration,
                                                   // the wildcard IP lets us bind to all network interfaces
                                                   NetUtils.getWildcardIPAddress, externalPort, externalHostname, externalPort)

            remoteConfig.withFallback(defaultConfig)
        }

        case None =>
        defaultConfig
    }
}

Akka配置

从AkkaConfig中我们可以找到Akka基础配置和远程调用配置模版。这些模版非常详细,它使用了Scala的String interpolation方式将配置值填充入模版。

基本配置

Akka基本配置生成逻辑位于getBasicAkkaConfig方法中。组装的配置文件如下:

val config =
  s"""
    |akka {
    | daemonic = off
    |
    | loggers = ["akka.event.slf4j.Slf4jLogger"]
    | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter"
    | log-config-on-start = off
    | logger-startup-timeout = 30s
    |
    | jvm-exit-on-fatal-error = $jvmExitOnFatalError
    |
    | serialize-messages = off
    |
    | loglevel = $logLevel
    | stdout-loglevel = OFF
    |
    | log-dead-letters = $logLifecycleEvents
    | log-dead-letters-during-shutdown = $logLifecycleEvents
    |
    | actor {
    |   guardian-supervisor-strategy = $supervisorStrategy
    |
    |   warn-about-java-serializer-usage = off
    |
    |   default-dispatcher {
    |     throughput = $akkaThroughput
    |   }
    |
    |   supervisor-dispatcher {
    |     type = Dispatcher
    |     executor = "thread-pool-executor"
    |     thread-pool-executor {
    |       core-pool-size-min = 1
    |       core-pool-size-max = 1
    |     }
    |   }
    | }
    |}
  """.stripMargin

远程配置

Akka的远程配置组装方法位于getRemoteAkkaConfig中。这里使用了akka-remote模块实现远程调用。

val configString =
s"""
         |akka {
         |  actor {
         |    provider = "akka.remote.RemoteActorRefProvider"
         |  }
         |
         |  remote {
         |    startup-timeout = $startupTimeout
         |
         |    transport-failure-detector{
         |      acceptable-heartbeat-pause = $transportHeartbeatPause
         |      heartbeat-interval = $transportHeartbeatInterval
         |      threshold = $transportThreshold
         |    }
         |
         |    netty {
         |      tcp {
         |        transport-class = "akka.remote.transport.netty.NettyTransport"
         |        port = $externalPort
         |        bind-port = $port
         |        connection-timeout = $akkaTCPTimeout
         |        maximum-frame-size = $akkaFramesize
         |        tcp-nodelay = on
         |
         |        client-socket-worker-pool {
         |          pool-size-min = $clientSocketWorkerPoolPoolSizeMin
         |          pool-size-max = $clientSocketWorkerPoolPoolSizeMax
         |          pool-size-factor = $clientSocketWorkerPoolPoolSizeFactor
         |        }
         |
         |        server-socket-worker-pool {
         |          pool-size-min = $serverSocketWorkerPoolPoolSizeMin
         |          pool-size-max = $serverSocketWorkerPoolPoolSizeMax
         |          pool-size-factor = $serverSocketWorkerPoolPoolSizeFactor
         |        }
         |      }
         |    }
         |
         |    log-remote-lifecycle-events = $logLifecycleEvents
         |
         |    retry-gate-closed-for = ${retryGateClosedFor + " ms"}
         |  }
         |}
       """.stripMargin

val hostnameConfigString =
s"""
         |akka {
         |  remote {
         |    netty {
         |      tcp {
         |        hostname = "$effectiveHostname"
         |        bind-hostname = "$bindAddress"
         |      }
         |    }
         |  }
         |}
       """.stripMargin

val sslConfigString = if (akkaEnableSSLConfig) {
    s"""
         |akka {
         |  remote {
         |
         |    enabled-transports = ["akka.remote.netty.ssl"]
         |
         |    netty {
         |
         |      ssl = $${akka.remote.netty.tcp}
         |
         |      ssl {
         |
         |        enable-ssl = $akkaEnableSSL
         |        ssl-engine-provider = org.apache.flink.runtime.akka.CustomSSLEngineProvider
         |        security {
         |          key-store = "$akkaSSLKeyStore"
         |          key-store-password = "$akkaSSLKeyStorePassword"
         |          key-password = "$akkaSSLKeyPassword"
         |          trust-store = "$akkaSSLTrustStore"
         |          trust-store-password = "$akkaSSLTrustStorePassword"
         |          protocol = $akkaSSLProtocol
         |          enabled-algorithms = $akkaSSLAlgorithms
         |          random-number-generator = ""
         |          require-mutual-authentication = on
         |          cert-fingerprints = $akkaSSLCertFingerprints
         |        }
         |      }
         |    }
         |  }
         |}
       """.stripMargin
}else{
    ""
}

最后使用

ConfigFactory.parseString(configString + hostnameConfigString + sslConfigString).resolve()

将这3个配置项组装融合在一起,生成完整配置。

AkkaRpcService

前面章节讲述了RpcService的创建过程,接下来我们细看下AkkaRpcService

AkkaRpcServiceRpcService的唯一实现类。它除了持有AkkaSystem的引用外,还维护所有注册了的RpcEndpoint的引用,为每个RpcEndpoint分配一个ActerRef并保存他们的对应关系。

@GuardedBy("lock")
private final Map actors = new HashMap<>(4);

我们从构造函数开始分析:

@VisibleForTesting
public AkkaRpcService(
        final ActorSystem actorSystem, final AkkaRpcServiceConfiguration configuration) {
    this.actorSystem = checkNotNull(actorSystem, "actor system");
    this.configuration = checkNotNull(configuration, "akka rpc service configuration");

    Address actorSystemAddress = AkkaUtils.getAddress(actorSystem);

    // 获取ActorSystem的host地址
    if (actorSystemAddress.host().isDefined()) {
        address = actorSystemAddress.host().get();
    } else {
        address = "";
    }

    // 获取端口号,如果没有配置,返回-1
    if (actorSystemAddress.port().isDefined()) {
        port = (Integer) actorSystemAddress.port().get();
    } else {
        port = -1;
    }

    // 是否捕捉Ask操作的调用栈
    captureAskCallstacks = configuration.captureAskCallStack();

    // ActorSystem scheduler的一个包装类
    internalScheduledExecutor = new ActorSystemScheduledExecutorAdapter(actorSystem);

    // RPC服务停机之后的异步回调
    terminationFuture = new CompletableFuture<>();

    // 标记RPC服务的状态,当前是正在运行
    stopped = false;

    // 启动SupervisorActor
    supervisor = startSupervisorActor();
}

注意到构造函数的最后一行启动SupervisorActor。一个AkkaRpcService可以连接提供给多个RpcEndpoint使用,它为每个RpcEndpoint创建一个Actor,这些Actor都是SupervisorActor的子Actor

private Supervisor startSupervisorActor() {
    final ExecutorService terminationFutureExecutor =
        Executors.newSingleThreadExecutor(
        new ExecutorThreadFactory(
            "AkkaRpcService-Supervisor-Termination-Future-Executor"));
    // 创建一个SupervisorActor
    final ActorRef actorRef =
        SupervisorActor.startSupervisorActor(actorSystem, terminationFutureExecutor);

    // 将SupervisorActor和terminationFutureExecutor包装后返回
    return Supervisor.create(actorRef, terminationFutureExecutor);
}

startServer

startServer方法在创建RpcEndpoint的时候调用:

protected RpcEndpoint(final RpcService rpcService) {
    this(rpcService, UUID.randomUUID().toString());
}
protected RpcEndpoint(final RpcService rpcService, final String endpointId) {
    this.rpcService = checkNotNull(rpcService, "rpcService");
    this.endpointId = checkNotNull(endpointId, "endpointId");
    // 在创建RpcEndpoint的时候需要传入RpcService,同时调用了它的startServer方法
    this.rpcServer = rpcService.startServer(this);

    this.mainThreadExecutor = new MainThreadExecutor(rpcServer, this::validateRunsInMainThread);
}

接下来我们分析AkkaRpcServerstartServer方法:

@Override
public  RpcServer startServer(C rpcEndpoint) {
    checkNotNull(rpcEndpoint, "rpc endpoint");

    // 向SupervisorActor注册,生成一个新的Actor
    final SupervisorActor.ActorRegistration actorRegistration =
            registerAkkaRpcActor(rpcEndpoint);
    // 获取这个Actor
    final ActorRef actorRef = actorRegistration.getActorRef();
    // 获取这个Actor终止运行时候的CompletableFuture
    final CompletableFuture actorTerminationFuture =
            actorRegistration.getTerminationFuture();

    LOG.info(
            "Starting RPC endpoint for {} at {} .",
            rpcEndpoint.getClass().getName(),
            actorRef.path());

    // 获取akka地址和hostname
    final String akkaAddress = AkkaUtils.getAkkaURL(actorSystem, actorRef);
    final String hostname;
    Option host = actorRef.path().address().host();
    if (host.isEmpty()) {
        hostname = "localhost";
    } else {
        hostname = host.get();
    }

    // 这个方法获取rpcEndpoint所有实现的接口并保存入HashSet
    Set> implementedRpcGateways =
            new HashSet<>(RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass()));

    // 接口加入RpcServer,AkkaBasedEndpoint这两个
    implementedRpcGateways.add(RpcServer.class);
    implementedRpcGateways.add(AkkaBasedEndpoint.class);

    final InvocationHandler akkaInvocationHandler;

    // 下面是创建RpcServer代理类的过程
    
    // 根据RpcEndpoint的类型,使用不同的InvocationHandler
    if (rpcEndpoint instanceof FencedRpcEndpoint) {
        // a FencedRpcEndpoint needs a FencedAkkaInvocationHandler
        akkaInvocationHandler =
                new FencedAkkaInvocationHandler<>(
                        akkaAddress,
                        hostname,
                        actorRef,
                        configuration.getTimeout(),
                        configuration.getMaximumFramesize(),
                        actorTerminationFuture,
                        ((FencedRpcEndpoint) rpcEndpoint)::getFencingToken,
                        captureAskCallstacks);

        implementedRpcGateways.add(FencedMainThreadExecutable.class);
    } else {
        akkaInvocationHandler =
                new AkkaInvocationHandler(
                        akkaAddress,
                        hostname,
                        actorRef,
                        configuration.getTimeout(),
                        configuration.getMaximumFramesize(),
                        actorTerminationFuture,
                        captureAskCallstacks);
    }

    // Rather than using the System ClassLoader directly, we derive the ClassLoader
    // from this class . That works better in cases where Flink runs embedded and all Flink
    // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
    ClassLoader classLoader = getClass().getClassLoader();

    // 使用JDK代理方式创建一个RpcServer
    // 调用RpcServer实际上执行的是akkaInvocationHandler
    // 接下来的一章专门分析
    @SuppressWarnings("unchecked")
    RpcServer server =
            (RpcServer)
                    Proxy.newProxyInstance(
                            classLoader,
                            implementedRpcGateways.toArray(
                                    new Class[implementedRpcGateways.size()]),
                            akkaInvocationHandler);

    return server;
}

registerAkkaRpcActor方法从SupervisorActor创建出子Actor,类型为FencedAkkaRpcActor或者是AkkaRpcActor

private 
    SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) {
    final Class akkaRpcActorType;

    // 判断RpcEndpoint的类型,创建对应的Actor
    if (rpcEndpoint instanceof FencedRpcEndpoint) {
        akkaRpcActorType = FencedAkkaRpcActor.class;
    } else {
        akkaRpcActorType = AkkaRpcActor.class;
    }

    synchronized (lock) {
        // 先检查RpcService必须为停止状态
        checkState(!stopped, "RpcService is stopped");

        // 从SupervisorActor创建一个新的Actor
        final SupervisorActor.StartAkkaRpcActorResponse startAkkaRpcActorResponse =
            SupervisorActor.startAkkaRpcActor(
            // 获取SupervisorActor
            supervisor.getActor(),
            // 传入Actor构造工厂方法
            actorTerminationFuture ->
            Props.create(
                akkaRpcActorType,
                rpcEndpoint,
                actorTerminationFuture,
                getVersion(),
                configuration.getMaximumFramesize()),
            //最后是endpoint的id
            rpcEndpoint.getEndpointId());

        // 为actorRegistration绑定异常响应
        final SupervisorActor.ActorRegistration actorRegistration =
            startAkkaRpcActorResponse.orElseThrow(
            cause ->
            new AkkaRpcRuntimeException(
                String.format(
                    "Could not create the %s for %s.",
                    AkkaRpcActor.class.getSimpleName(),
                    rpcEndpoint.getEndpointId()),
                cause));

        // 将这个新创建的actor和关联的rpcEndpoint对应关系保存
        actors.put(actorRegistration.getActorRef(), rpcEndpoint);

        return actorRegistration;
    }
}

接着分析SupervisorActor.startAkkaRpcActor,即创建Actor的方法。

public static StartAkkaRpcActorResponse startAkkaRpcActor(
    ActorRef supervisor, StartAkkaRpcActor.PropsFactory propsFactory, String endpointId) {
    // 调用SupervisorActor并同步等待结果
    // 发送给SupervisorActor的消息通过createStartAkkaRpcActorMessage创建
    return Patterns.ask(
        supervisor,
        createStartAkkaRpcActorMessage(propsFactory, endpointId),
        RpcUtils.INF_DURATION)
        .toCompletableFuture()
        .thenApply(SupervisorActor.StartAkkaRpcActorResponse.class::cast)
        .join();
}

接着需要跟踪SupervisorActor如何响应创建actor的请求。

@Override
public Receive createReceive() {
    return receiveBuilder()
        .match(StartAkkaRpcActor.class, this::createStartAkkaRpcActorMessage)
        .matchAny(this::handleUnknownMessage)
        .build();
}

SupervisiorActor接收到StartAkkaRpcActor类型的message,调用createStartAkkaRpcActorMessage处理方法。代码如下:

private void createStartAkkaRpcActorMessage(StartAkkaRpcActor startAkkaRpcActor) {
    // 从接收到的消息中取回endpoint id
    final String endpointId = startAkkaRpcActor.getEndpointId();
    // 创建AkkaRpcActorRegistration
    final AkkaRpcActorRegistration akkaRpcActorRegistration =
        new AkkaRpcActorRegistration(endpointId);

    // 从接收到的消息中取回属性工厂方法,创建出Props
    // 这样做的目的是方便操作InternalTerminationFuture
    // 属性构建逻辑在RpcService中指定但是InternalTerminationFuture不会暴露给RpcService
    final Props akkaRpcActorProps =
        startAkkaRpcActor
        .getPropsFactory()
        .create(akkaRpcActorRegistration.getInternalTerminationFuture());

    LOG.debug(
        "Starting {} with name {}.",
        akkaRpcActorProps.actorClass().getSimpleName(),
        endpointId);

    try {
        // 创建出子actor,名称为endpoint的id
        final ActorRef actorRef = getContext().actorOf(akkaRpcActorProps, endpointId);

        // 保存这个actor
        registeredAkkaRpcActors.put(actorRef, akkaRpcActorRegistration);

        // 将actor创建成功的消息发送回调用端
        // 创建成功的actor也一并返回
        getSender()
            .tell(
            StartAkkaRpcActorResponse.success(
                ActorRegistration.create(
                    actorRef,
                    akkaRpcActorRegistration
                    .getExternalTerminationFuture())),
            getSelf());
    } catch (AkkaException akkaException) {
        getSender().tell(StartAkkaRpcActorResponse.failure(akkaException), getSelf());
    }
}

connect方法

在这一节分析connect方法。该方法使用指定的地址,连接到远程Rpc服务。第二个参数class的含义为RpcGateway的类型。RpcGateway是和远程ActorSystem通信的途径。建立连接之后,调用端可以使用返回的C类型对象远程调用actor。

@Override
public  CompletableFuture connect(
    final String address, final Class clazz) {

    return connectInternal(
        address,
        clazz,
        // 这里是一个工厂方法
        // 从actorRef创建出InvocationHandler
        // 在connectInternal方法中使用这个工厂方法
        (ActorRef actorRef) -> {
            // 获取actor的地址和hostname
            Tuple2 addressHostname = extractAddressHostname(actorRef);

            // 和前面一样,也是创建出一个AkkaInvocationHandler对象
            return new AkkaInvocationHandler(
                addressHostname.f0,
                addressHostname.f1,
                actorRef,
                configuration.getTimeout(),
                configuration.getMaximumFramesize(),
                null,
                captureAskCallstacks);
        });
}

connectInternal方法内容如下:

private  CompletableFuture connectInternal(
    final String address,
    final Class clazz,
    Function invocationHandlerFactory) {
    checkState(!stopped, "RpcService is stopped");

    LOG.debug(
        "Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.",
        address,
        clazz.getName());

    // 从远程地址获取ActorRef,超时时间在AkkaRpcServiceConfiguration中
    final CompletableFuture actorRefFuture = resolveActorAddress(address);

    // 获取到ActorRef后执行
    final CompletableFuture handshakeFuture =
        actorRefFuture.thenCompose(
        (ActorRef actorRef) ->
        FutureUtils.toJava(
            // 发送RemoteHandshakeMessage给远端
            // 这是握手步骤
            Patterns.ask(
                actorRef,
                new RemoteHandshakeMessage(
                    clazz, getVersion()),
                configuration.getTimeout().toMilliseconds())
            .mapTo(
                ClassTag$.MODULE$
                .apply(
                    HandshakeSuccessMessage
                    .class))));

    // 返回一个创建RpcGateway的CompletableFuture
    return actorRefFuture.thenCombineAsync(
        handshakeFuture,
        (ActorRef actorRef, HandshakeSuccessMessage ignored) -> {
            // 在握手操作完毕后执行
            // 使用前面所述的方法参数传入的invocationHandlerFactory,使用actorRef创建出一个AkkaInvocationHandler
            InvocationHandler invocationHandler = invocationHandlerFactory.apply(actorRef);

            // Rather than using the System ClassLoader directly, we derive the ClassLoader
            // from this class . That works better in cases where Flink runs embedded and
            // all Flink
            // code is loaded dynamically (for example from an OSGI bundle) through a custom
            // ClassLoader
            ClassLoader classLoader = getClass().getClassLoader();

            // 创建出代理类,类型转换为C类型
            // 使用JDK代理
            @SuppressWarnings("unchecked")
            C proxy =
                (C)
                Proxy.newProxyInstance(
                classLoader, new Class[] {clazz}, invocationHandler);

            return proxy;
        },
        actorSystem.dispatcher());
}

通过上面的分析我们了解到最终创建出的无论是RpcServer还是C extends RpcGateway,都是使用了JDK的动态代理。真实的执行逻辑都在invocationHandler中。下一节我们分析AkkaInvocationHandler

AkkaInvocationHandler

接上一节,RpcServer创建出的远程调用端实际上为JDK代理类,它的真实执行逻辑在AkkaInvocationHandler中。关于Java动态代理详细介绍参见:Java 动态代理

发起RPC远程方法调用

invoke方法根据声明method的位置,决定调用AkkaInvocationHandler中的方法还是走远程调用。详细分析如下:

@Override
public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
    // 获取声明该方法的class或接口
    Class declaringClass = method.getDeclaringClass();

    Object result;
    // 如果method在这几个接口或类中定义,本地调用AkkaInvocationHandler的同名方法
    if (declaringClass.equals(AkkaBasedEndpoint.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)) {
        // 不支持FencedRpcGateway中的方法
        throw new UnsupportedOperationException(
            "AkkaInvocationHandler does not support the call FencedRpcGateway#"
            + method.getName()
            + ". This indicates that you retrieved a FencedRpcGateway without specifying a "
            + "fencing token. Please use RpcService#connect(RpcService, F, Time) with F being the fencing token to "
            + "retrieve a properly FencedRpcGateway.");
    } else {
        // 如果method不属于上面所有的接口,走RPC远程调用
        result = invokeRpc(method, args);
    }

    return result;
}

我们继续分析RPC调用逻辑。invokeRpc方法内容和分析如下所示:

private Object invokeRpc(Method method, Object[] args) throws Exception {
    String methodName = method.getName();
    Class[] parameterTypes = method.getParameterTypes();
    // 获取所有的参数附带的注解,以二维数组形式返回
    Annotation[][] parameterAnnotations = method.getParameterAnnotations();
    
    // 从方法参数中获取RpcTimeout注解标记的参数,并读取超时时间
    Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);

    // 创建Actor需要发送的信息,包含method调用的必要参数
    final RpcInvocation rpcInvocation =
        createRpcInvocationMessage(methodName, parameterTypes, args);

    // 获取方法的返回类型
    Class returnType = method.getReturnType();

    final Object result;

    // 如果方法无返回值,直接发送给远程Actor
    if (Objects.equals(returnType, Void.TYPE)) {
        tell(rpcInvocation);

        result = null;
    } else {
        // Capture the call stack. It is significantly faster to do that via an exception than
        // via Thread.getStackTrace(), because exceptions lazily initialize the stack trace,
        // initially only
        // capture a lightweight native pointer, and convert that into the stack trace lazily
        // when needed.
        // 获取CallStack,用于拼装stackstrace,节省调用时间
        final Throwable callStackCapture = captureAskCallStack ? new Throwable() : null;

        // execute an asynchronous call
        // 发送rpcInvocation给远程Actor并等待回复,超时时间为futureTimeout
        final CompletableFuture resultFuture = ask(rpcInvocation, futureTimeout);

        final CompletableFuture completableFuture = new CompletableFuture<>();
        // 获取远程调用结果,如有需要则反序列化(返回值为AkkaRpcSerializedValue类型)
        resultFuture.whenComplete(
            (resultValue, failure) -> {
                if (failure != null) {
                    completableFuture.completeExceptionally(
                        resolveTimeoutException(failure, callStackCapture, method));
                } else {
                    completableFuture.complete(
                        deserializeValueIfNeeded(resultValue, method));
                }
            });

        // 如果返回值为CompletableFuture类型,直接返回
        // 否则从completableFuture阻塞等待获取值之后返回
        if (Objects.equals(returnType, CompletableFuture.class)) {
            result = completableFuture;
        } else {
            try {
                result =
                    completableFuture.get(futureTimeout.getSize(), futureTimeout.getUnit());
            } catch (ExecutionException ee) {
                throw new RpcException(
                    "Failure while obtaining synchronous RPC result.",
                    ExceptionUtils.stripExecutionException(ee));
            }
        }
    }

    return result;
}
 
 

接下来的问题是我们如何让被调用端知道我们希望调用的方法是什么,参数是什么。这些关键信息包含在createRpcInvocationMessage方法中。内容如下:

protected RpcInvocation createRpcInvocationMessage(
    final String methodName, final Class[] parameterTypes, final Object[] args)
    throws IOException {
    final RpcInvocation rpcInvocation;

    // 如果是本地调用
    // 创建LocalRpcInvocation
    if (isLocal) {
        rpcInvocation = new LocalRpcInvocation(methodName, parameterTypes, args);
    } else {
        try {
            // 否则创建remoteRpcInvocation
            RemoteRpcInvocation remoteRpcInvocation =
                new RemoteRpcInvocation(methodName, parameterTypes, args);

            // 检查序列化之后的远程方法调用对象是否超过了最大帧大小
            // RemoteRpcInvocation包含了序列化方法,这样才能通过Akka发送
            // 如果超过不让发送
            if (remoteRpcInvocation.getSize() > maximumFramesize) {
                throw new IOException(
                    String.format(
                        "The rpc invocation size %d exceeds the maximum akka framesize.",
                        remoteRpcInvocation.getSize()));
            } else {
                rpcInvocation = remoteRpcInvocation;
            }
        } catch (IOException e) {
            LOG.warn(
                "Could not create remote rpc invocation message. Failing rpc invocation because...",
                e);
            throw e;
        }
    }

    return rpcInvocation;
}

RpcInvocation

RpcInvocation是RPC调用发给远程Actor的信息载体。上面的LocalRpcInvocationRemoteRpcInvocation

它封装RPC远程方法调用的必要参数,分别为方法名,参数类型列表和参数列表。接口代码如下所示:

public interface RpcInvocation {

    /**
     * Returns the method's name.
     *
     * @return Method name
     * @throws IOException if the rpc invocation message is a remote message and could not be
     *     deserialized
     * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
     *     serialized classes which cannot be found on the receiving side
     */
    String getMethodName() throws IOException, ClassNotFoundException;

    /**
     * Returns the method's parameter types
     *
     * @return Method's parameter types
     * @throws IOException if the rpc invocation message is a remote message and could not be
     *     deserialized
     * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
     *     serialized classes which cannot be found on the receiving side
     */
    Class[] getParameterTypes() throws IOException, ClassNotFoundException;

    /**
     * Returns the arguments of the remote procedure call
     *
     * @return Arguments of the remote procedure call
     * @throws IOException if the rpc invocation message is a remote message and could not be
     *     deserialized
     * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
     *     serialized classes which cannot be found on the receiving side
     */
    Object[] getArgs() throws IOException, ClassNotFoundException;
}

LocalRpcInvocation为它的简单实现,没有添加其他逻辑,这里不再分析代码。

需要重点关注的是RemoteRpcInvocationRemoteRpcInvocation是支持序列化的,这样才能够将这个message发送给远程actor。RemoteRpcInvocation的信息载体为内部类MethodInvocation。同样具有Method反射调用三要素

private String methodName;
private Class[] parameterTypes;
private Object[] args;

我们接着分析下它是如何把这三个参数序列化和反序列化的。

private void writeObject(ObjectOutputStream oos) throws IOException {
    // 首先写入方法名
    oos.writeUTF(methodName);

    // 写入参数个数
    oos.writeInt(parameterTypes.length);

    // 逐个写入参数类型
    for (Class parameterType : parameterTypes) {
        oos.writeObject(parameterType);
    }

    if (args != null) {
        // 如果有参数列表,先写入true
        oos.writeBoolean(true);

        // 逐个写入参数对象
        for (int i = 0; i < args.length; i++) {
            try {
                oos.writeObject(args[i]);
            } catch (IOException e) {
                throw new IOException(
                    "Could not serialize "
                    + i
                    + "th argument of method "
                    + methodName
                    + ". This indicates that the argument type "
                    + args.getClass().getName()
                    + " is not serializable. Arguments have to "
                    + "be serializable for remote rpc calls.",
                    e);
            }
        }
    } else {
        // 如果没有参数列表,写入false
        oos.writeBoolean(false);
    }
}

// 和序列化的逻辑相对应
private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
    // 读取方法名
    methodName = ois.readUTF();

    // 读取参数列表长度
    int length = ois.readInt();

    parameterTypes = new Class[length];

    // 读取参数类型
    for (int i = 0; i < length; i++) {
        try {
            parameterTypes[i] = (Class) ois.readObject();
        } catch (IOException e) {
            StringBuilder incompleteMethod = getIncompleteMethodString(i, 0);
            throw new IOException(
                "Could not deserialize "
                + i
                + "th parameter type of method "
                + incompleteMethod
                + '.',
                e);
        } catch (ClassNotFoundException e) {
            // note: wrapping this CNFE into another CNFE does not overwrite the Exception
            //       stored in the ObjectInputStream (see ObjectInputStream#readSerialData)
            // -> add a suppressed exception that adds a more specific message
            StringBuilder incompleteMethod = getIncompleteMethodString(i, 0);
            e.addSuppressed(
                new ClassNotFoundException(
                    "Could not deserialize "
                    + i
                    + "th "
                    + "parameter type of method "
                    + incompleteMethod
                    + ". This indicates that the parameter "
                    + "type is not part of the system class loader."));
            throw e;
        }
    }

    // 读取boolean判断调用时候是否传入了参数
    boolean hasArgs = ois.readBoolean();

    if (hasArgs) {
        args = new Object[length];

        // 如果有参数,逐个读取
        for (int i = 0; i < length; i++) {
            try {
                args[i] = ois.readObject();
            } catch (IOException e) {
                StringBuilder incompleteMethod = getIncompleteMethodString(length, i);
                throw new IOException(
                    "Could not deserialize "
                    + i
                    + "th argument of method "
                    + incompleteMethod
                    + '.',
                    e);
            } catch (ClassNotFoundException e) {
                // note: wrapping this CNFE into another CNFE does not overwrite the
                // Exception
                //       stored in the ObjectInputStream (see
                // ObjectInputStream#readSerialData)
                // -> add a suppressed exception that adds a more specific message
                StringBuilder incompleteMethod = getIncompleteMethodString(length, i);
                e.addSuppressed(
                    new ClassNotFoundException(
                        "Could not deserialize "
                        + i
                        + "th "
                        + "argument of method "
                        + incompleteMethod
                        + ". This indicates that the argument "
                        + "type is not part of the system class loader."));
                throw e;
            }
        }
    } else {
        // 如果无参数,args赋值为null
        args = null;
    }
}

发送RemoteRpcInvocation给远端的时候,实际包含的内容是serializedMethodInvocation。它包含的是字节数组类型,在创建serializedMethodInvocation的时候,会使用InstantiationUtil.deserializeObjectMethodInvocation序列化。

// Serialized invocation data
private SerializedValue serializedMethodInvocation;

// Transient field which is lazily initialized upon first access to the invocation data
private transient RemoteRpcInvocation.MethodInvocation methodInvocation;

AkkaRpcActor

这一章我们开始分析actor。如上面所述,在Flink中所有的AkkaRpcActor都是SupervisouActor的子actor,AkkaRpcActorRpcServicestartServer方法创建出。

还有一种FencedAkkaRpcActor,它为了避免脑裂而设计。使用FencingToken来标记可接受哪个发送者的消息。比如说JobManager配置了HA,存在两个JobManager:JobManager1和JobManager2。一开始JobManager1为leader状态,FencingToken被设置为JobManager1的ID,远端都只接收JobManager1发来的消息。突然JobManager1崩溃,JobManager2获得了leader,设置FencingToken为JobManager2的ID,这时候远端都只接收JobManager2发来的消息。然而,又过了一会儿,JobManager1恢复,在它得知自己失去leader状态之前,仍会发送消息,因为FencingToken已经更改,校验失败,远端拒绝接受JobManager1发来的消息,从而避免了脑裂问题。

除了Fencing机制外,FencedAkkaRpcActorAkkaRpcActor的逻辑相同。下面的分析都以AkkaRpcActor为准。

接受RPC调用

AkkaRpcActor接收message的处理方法如下:

@Override
public Receive createReceive() {
    return ReceiveBuilder.create()
        .match(RemoteHandshakeMessage.class, this::handleHandshakeMessage)
        .match(ControlMessages.class, this::handleControlMessage)
        .matchAny(this::handleMessage)
        .build();
}

这个方法很容易理解,如果Message类型为RemoteHandshakeMessage,调用handleHandshakeMessage方法处理(建立连接时的握手信息),如果类型为ControlMessages,使用handleControlMessage处理(改变actor的启动停止状态等),其他的message类型都使用handleMessage方法处理。接下来我们重点关注处理消息的handleMessage方法。

private void handleMessage(final Object message) {
    if (state.isRunning()) {
        // 确保handleRpcMessage方法没有被多线程调用
        mainThreadValidator.enterMainThread();

        try {
            // 处理Rpc消息,下面分析
            handleRpcMessage(message);
        } finally {
            mainThreadValidator.exitMainThread();
        }
    } else {
        log.info(
            "The rpc endpoint {} has not been started yet. Discarding message {} until processing is started.",
            rpcEndpoint.getClass().getName(),
            message.getClass().getName());

        sendErrorIfSender(
            new AkkaRpcException(
                String.format(
                    "Discard message, because the rpc endpoint %s has not been started yet.",
                    rpcEndpoint.getAddress())));
    }
}

handleRpcMessage

handleRpcMessage方法内容如下所示:

protected void handleRpcMessage(Object message) {
    if (message instanceof RunAsync) {
        handleRunAsync((RunAsync) message);
    } else if (message instanceof CallAsync) {
        handleCallAsync((CallAsync) message);
    } else if (message instanceof RpcInvocation) {
        handleRpcInvocation((RpcInvocation) message);
    } else {
        log.warn(
            "Received message of unknown type {} with value {}. Dropping this message!",
            message.getClass().getName(),
            message);

        sendErrorIfSender(
            new AkkaUnknownMessageException(
                "Received unknown message "
                + message
                + " of type "
                + message.getClass().getSimpleName()
                + '.'));
    }
}

这个方法中进一步根据message的类型拆分逻辑,调用相应的处理方法。message可能为RunAsync(异步无返回值调用),CallAsync(异步有返回值调用)和RpcInvocation(RPC方法调用)。我们重点关注的是RPC方法调用。

handleRpcInvocation

private void handleRpcInvocation(RpcInvocation rpcInvocation) {
    Method rpcMethod = null;

    try {
        // 获取方法名和参数列表
        String methodName = rpcInvocation.getMethodName();
        Class[] parameterTypes = rpcInvocation.getParameterTypes();

        // 从rpcEndpoint中查找匹配的方法
        // 这里的RpcEndpoint为T类型,并非RpcEndpoint这个类
        rpcMethod = lookupRpcMethod(methodName, parameterTypes);
        // 如果出错,将错误信息回送给调用者
    } catch (ClassNotFoundException e) {
        log.error("Could not load method arguments.", e);

        RpcConnectionException rpcException =
            new RpcConnectionException("Could not load method arguments.", e);
        getSender().tell(new Status.Failure(rpcException), getSelf());
    } catch (IOException e) {
        log.error("Could not deserialize rpc invocation message.", e);

        RpcConnectionException rpcException =
            new RpcConnectionException("Could not deserialize rpc invocation message.", e);
        getSender().tell(new Status.Failure(rpcException), getSelf());
    } catch (final NoSuchMethodException e) {
        log.error("Could not find rpc method for rpc invocation.", e);

        RpcConnectionException rpcException =
            new RpcConnectionException("Could not find rpc method for rpc invocation.", e);
        getSender().tell(new Status.Failure(rpcException), getSelf());
    }

    if (rpcMethod != null) {
        try {
            // this supports declaration of anonymous classes
            // 设置方法可以被访问
            rpcMethod.setAccessible(true);

            // 如果方法没有返回值,直接调用
            if (rpcMethod.getReturnType().equals(Void.TYPE)) {
                // No return value to send back
                rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
            } else {
                // 如果有返回值,调用后将返回值获取
                final Object result;
                try {
                    result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
                } catch (InvocationTargetException e) {
                    log.debug(
                        "Reporting back error thrown in remote procedure {}", rpcMethod, e);

                    // tell the sender about the failure
                    getSender().tell(new Status.Failure(e.getTargetException()), getSelf());
                    return;
                }

                final String methodName = rpcMethod.getName();

                // 判断返回值是不是CompletableFuture,如果是的话说明结果可以异步返回
                if (result instanceof CompletableFuture) {
                    final CompletableFuture responseFuture = (CompletableFuture) result;
                    sendAsyncResponse(responseFuture, methodName);
                } else {
                    // 否则同步返回调用结果
                    sendSyncResponse(result, methodName);
                }
            }
        } catch (Throwable e) {
            log.error("Error while executing remote procedure call {}.", rpcMethod, e);
            // tell the sender about the failure
            getSender().tell(new Status.Failure(e), getSelf());
        }
    }
}

sendAsyncResponse方法异步返回调用结果,内容如下所示:

private void sendAsyncResponse(CompletableFuture asyncResponse, String methodName) {
    final ActorRef sender = getSender();
    Promise.DefaultPromise promise = new Promise.DefaultPromise<>();

    FutureUtils.assertNoException(
        asyncResponse.handle(
            (value, throwable) -> {
                // 检查异步结果是否有异常
                if (throwable != null) {
                    promise.failure(throwable);
                } else {
                    if (isRemoteSender(sender)) {
                        // 如果是远端发送的调用
                        // 将返回结果序列化
                        Either
                            serializedResult =
                            serializeRemoteResultAndVerifySize(
                            value, methodName);

                        // 如果序列化成功,调用 promise.success,否则调用failure
                        if (serializedResult.isLeft()) {
                            promise.success(serializedResult.left());
                        } else {
                            promise.failure(serializedResult.right());
                        }
                    } else {
                        // 如果是本地调用,不需序列化
                        promise.success(new Status.Success(value));
                    }
                }

                // consume the provided throwable
                return null;
            }));

    // 发送future到调用端
    Patterns.pipe(promise.future(), getContext().dispatcher()).to(sender);
}
 
 

sendSyncResponse方法为同步返回结果。内容和上面类似,只是不用在返回future给调用端。

private void sendSyncResponse(Object response, String methodName) {
    if (isRemoteSender(getSender())) {
        Either serializedResult =
            serializeRemoteResultAndVerifySize(response, methodName);

        if (serializedResult.isLeft()) {
            getSender().tell(new Status.Success(serializedResult.left()), getSelf());
        } else {
            getSender().tell(new Status.Failure(serializedResult.right()), getSelf());
        }
    } else {
        getSender().tell(new Status.Success(response), getSelf());
    }
}

RpcEndpoint

最后我们分析到了RpcEndpoint。它是提供RPC服务的被调用端。需要接收远程调用的类例如JobMaster,TaskExecutor,Dispatcher等都需要继承这个RpcEndpoint

除此之外,RpcEndpoint还可以在Actor主线程池定时执行Runnable。安排定时任务的方法如下:

protected void scheduleRunAsync(Runnable runnable, Time delay) {
    scheduleRunAsync(runnable, delay.getSize(), delay.getUnit());
}

protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
    rpcServer.scheduleRunAsync(runnable, unit.toMillis(delay));
}

rpcServerscheduleRunAsync方法位于AkkaInvocationHandler中:

@Override
public void scheduleRunAsync(Runnable runnable, long delayMillis) {
    checkNotNull(runnable, "runnable");
    checkArgument(delayMillis >= 0, "delay must be zero or greater");

    if (isLocal) {
        long atTimeNanos = delayMillis == 0 ? 0 : System.nanoTime() + (delayMillis * 1_000_000);
        tell(new RunAsync(runnable, atTimeNanos));
    } else {
        throw new RuntimeException(
            "Trying to send a Runnable to a remote actor at "
            + rpcEndpoint.path()
            + ". This is not supported.");
    }
}

最后是tell方法,把message发送给了rpcEndpoint对应的actor。

protected void tell(Object message) {
    rpcEndpoint.tell(message, ActorRef.noSender());
}

Actor处理RunAsync的方法为handleRunAsync。如下所示:

private void handleRunAsync(RunAsync runAsync) {
    final long timeToRun = runAsync.getTimeNanos();
    final long delayNanos;

    // 如果没配置运行时刻,或者是运行时刻已过
    if (timeToRun == 0 || (delayNanos = timeToRun - System.nanoTime()) <= 0) {
        // run immediately
        // 需要立刻执行
        try {
            runAsync.getRunnable().run();
        } catch (Throwable t) {
            log.error("Caught exception while executing runnable in main thread.", t);
            ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
        }
    } else {
        // schedule for later. send a new message after the delay, which will then be
        // immediately executed
        // 否则需要延迟执行
        FiniteDuration delay = new FiniteDuration(delayNanos, TimeUnit.NANOSECONDS);
        RunAsync message = new RunAsync(runAsync.getRunnable(), timeToRun);

        final Object envelopedSelfMessage = envelopeSelfMessage(message);

        // 使用ActorSystem的scheduler安排一个延时执行任务
        getContext()
            .system()
            .scheduler()
            .scheduleOnce(
            delay,
            getSelf(),
            envelopedSelfMessage,
            getContext().dispatcher(),
            ActorRef.noSender());
    }
}

本博客为作者原创,欢迎大家参与讨论和批评指正。如需转载请注明出处。

你可能感兴趣的:(Flink 源码之RPC调用)