Flink RPC源码

DEMO

Flink RPC 众所周知底层是Akka,我先来写一个RPC 的用例

RPC Gateway

public interface HelloGateway extends RpcGateway {
    CompletableFuture<String> hello();
}

RPC Endpoint

也就是服务端执行的

public class HelloRpcEndpoint  extends RpcEndpoint implements HelloGateway{
    protected HelloRpcEndpoint(RpcService rpcService, String endpointId) {
        super(rpcService, endpointId);
    }

    @Override
    public CompletableFuture<String> hello() {
        return CompletableFuture.completedFuture("hello world");
    }
}

RPC Server

RPC Server 主要将我们定一个RPC Endpoint注册上,Client 请求这个网关的时候,对应执行Endpoint方法

public class ServerRPC {
    public static void main(String[] args) {
        ActorSystem defaultActorSystem = AkkaUtils.createActorSystem("flink", AkkaUtils.getAkkaConfig(new Configuration(), "localhost", 8888));
        RpcService rpcService = new AkkaRpcService(defaultActorSystem, AkkaRpcServiceConfiguration.defaultConfiguration());
        HelloRpcEndpoint helloRpcEndpoint = new HelloRpcEndpoint(rpcService, "hello");
        helloRpcEndpoint.start();
        System.out.println(helloRpcEndpoint.getAddress());
    }
}
  1. 拿到一个 ActorSystem,传入Default Config 、IP、Port
  2. new 一个 AkkaRpcService
  3. 注册 HelloRpcEndpoint
  4. 启用 HelloRpcEndpoint
  5. 输出HelloRpcEndpoint 的地址

RPC Client

public class ClientRPC {
    public static void main(String[] args) {
        // 可以看Server端的输出
        String address = "akka.tcp://flink@localhost:8888/user/rpc/hello";
        ActorSystem defaultActorSystem = AkkaUtils.createActorSystem("flink", AkkaUtils.getAkkaConfig(new Configuration(), "localhost", 9999));
        RpcService rpcService = new AkkaRpcService(defaultActorSystem, AkkaRpcServiceConfiguration.defaultConfiguration());
        CompletableFuture<HelloGateway> connect = rpcService.connect(address, HelloGateway.class);
        try {
            System.out.println(connect.get().hello().get());
        } catch (InterruptedException e) {
            e.printStackTrace();
        } catch (ExecutionException e) {
            e.printStackTrace();
        } finally {
            rpcService.stopService();
        }

    }
}
  1. 拿到一个 ActorSystem,传入Default Config 、IP、Port
  2. new 一个 AkkaRpcService
  3. RpcService 通过指定地址和Gateway class 拿到 HelloGateway
  4. 通过执行 HelloGateway 的方法远程调用。

源码

Server

我们根据以上的Demo 来跟源码就比较容易了。

public class ServerRPC {
    public static void main(String[] args) {
        ActorSystem defaultActorSystem = AkkaUtils.createActorSystem("flink", AkkaUtils.getAkkaConfig(new Configuration(), "localhost", 8888));
        RpcService rpcService = new AkkaRpcService(defaultActorSystem, AkkaRpcServiceConfiguration.defaultConfiguration());
        HelloRpcEndpoint helloRpcEndpoint = new HelloRpcEndpoint(rpcService, "hello");
        helloRpcEndpoint.start();
        System.out.println(helloRpcEndpoint.getAddress());
    }
}
  1. 创建 ActorSystem
  2. 构造RPC Service
  3. new Endpoint
  4. 启用 Endpoint

AkkaUtils#createActorSystem

  def createActorSystem(
      configuration: Configuration,
      listeningAddress: Option[(String, Int)])
    : ActorSystem = {
    val akkaConfig = getAkkaConfig(configuration, listeningAddress)
    createActorSystem(akkaConfig)
  }

  1. 根据config 和address 构造 Config
  2. 继续创建

AkkaUtils#createActorSystem

  def createActorSystem(actorSystemName: String, akkaConfig: Config): ActorSystem = {
    // Initialize slf4j as logger of Akka's Netty instead of java.util.logging (FLINK-1650)
    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory)
    RobustActorSystem.create(actorSystemName, akkaConfig)
  }
  1. 设置Log4j
  2. 继续创建

RobustActorSystem#internalApply

  def internalApply(
      name: String,
      setup: ActorSystemSetup,
      uncaughtExceptionHandler: Option[UncaughtExceptionHandler]): RobustActorSystem = {
    val bootstrapSettings = setup.get[BootstrapSetup]
    val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader())
    val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl))
    val defaultEC = bootstrapSettings.flatMap(_.defaultExecutionContext)

    new RobustActorSystem(
      name,
      appConfig,
      cl,
      defaultEC,
      None,
      setup,
      uncaughtExceptionHandler).start()
  }

RobustActorSystem 就是 继承的ActorSystemImpl,实际上已经创建了ActorSystem

AkkaRpcService#AkkaRpcService

public AkkaRpcService(ActorSystem actorSystem, AkkaRpcServiceConfiguration configuration) {
        this.actorSystem = (ActorSystem)Preconditions.checkNotNull(actorSystem, "actor system");
        this.configuration = (AkkaRpcServiceConfiguration)Preconditions.checkNotNull(configuration, "akka rpc service configuration");
        Address actorSystemAddress = AkkaUtils.getAddress(actorSystem);
        if (actorSystemAddress.host().isDefined()) {
            this.address = (String)actorSystemAddress.host().get();
        } else {
            this.address = "";
        }

        if (actorSystemAddress.port().isDefined()) {
            this.port = (Integer)actorSystemAddress.port().get();
        } else {
            this.port = -1;
        }

        this.captureAskCallstacks = configuration.captureAskCallStack();
        this.internalScheduledExecutor = new ActorSystemScheduledExecutorAdapter(actorSystem);
        this.terminationFuture = new CompletableFuture();
        this.stopped = false;
        this.supervisor = this.startSupervisorActor();
    }
  1. 传入 actorSystem 、configuration

RpcEndpoint#RpcEndpoint(RpcService, String)

    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);
    }

因为HelloWorldRpcEndpoint,是继承 RpcEndpoint,并且调用 super() 方法。所以直接看 RpcEndpoint 的构造方法。

  1. rpcService.startServer(this) 看方法是启动server。实际上将当前的Endpoint 注册到RpcServer 中
  2. 构建一个线程池

AkkaRpcService#startServer

    public <C extends RpcEndpoint & RpcGateway> RpcServer startServer(C rpcEndpoint) {
        checkNotNull(rpcEndpoint, "rpc endpoint");

        final SupervisorActor.ActorRegistration actorRegistration =
                registerAkkaRpcActor(rpcEndpoint);
        final ActorRef actorRef = actorRegistration.getActorRef();
        final CompletableFuture<Void> actorTerminationFuture =
                actorRegistration.getTerminationFuture();

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

        final String akkaAddress = AkkaUtils.getAkkaURL(actorSystem, actorRef);
        final String hostname;
        Option<String> host = actorRef.path().address().host();
        if (host.isEmpty()) {
            hostname = "localhost";
        } else {
            hostname = host.get();
        }

        Set<Class<?>> implementedRpcGateways =
                new HashSet<>(RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass()));

        implementedRpcGateways.add(RpcServer.class);
        implementedRpcGateways.add(AkkaBasedEndpoint.class);

        final InvocationHandler akkaInvocationHandler;

        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();

        @SuppressWarnings("unchecked")
        RpcServer server =
                (RpcServer)
                        Proxy.newProxyInstance(
                                classLoader,
                                implementedRpcGateways.toArray(
                                        new Class<?>[implementedRpcGateways.size()]),
                                akkaInvocationHandler);

        return server;
    }
  1. 注册 registerAkkaRpcActor

AkkaRpcService#registerAkkaRpcActor

    private final Map<ActorRef, RpcEndpoint> actors = new HashMap<>(4);
    
    
    private <C extends RpcEndpoint & RpcGateway>
            SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) {
        final Class<? extends AbstractActor> akkaRpcActorType;

        if (rpcEndpoint instanceof FencedRpcEndpoint) {
            akkaRpcActorType = FencedAkkaRpcActor.class;
        } else {
            akkaRpcActorType = AkkaRpcActor.class;
        }

        synchronized (lock) {
            checkState(!stopped, "RpcService is stopped");

            final SupervisorActor.StartAkkaRpcActorResponse startAkkaRpcActorResponse =
                    SupervisorActor.startAkkaRpcActor(
                            supervisor.getActor(),
                            actorTerminationFuture ->
                                    Props.create(
                                            akkaRpcActorType,
                                            rpcEndpoint,
                                            actorTerminationFuture,
                                            getVersion(),
                                            configuration.getMaximumFramesize()),
                            rpcEndpoint.getEndpointId());

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

            actors.put(actorRegistration.getActorRef(), rpcEndpoint);

            return actorRegistration;
        }
    }
  1. 判断是否是FencedAkkaRpcActor,我们使用的是AkkaRpcActor。注意哦,是 AkkaRpcActor
  2. 发送注册服务的请求,请求里面包含了 rpcEndpoint 等相关信息,其中会构造一个PropFactory 也是属性的工厂,熟悉工厂包含了很rpc信息。返回一个 StartAkkaRpcActorResponse
  3. StartAkkaRpcActorResponse 返回值里面有actorRegistration ,actorRegistration包含了当前 rpcEndpoint 的Acotr 的引用
  4. 将Actor 和 rpcEndpoint 放入Map中
  5. 返回 actorRegistration
SupervisorActor#startAkkaRpcActor
    public static StartAkkaRpcActorResponse startAkkaRpcActor(
            ActorRef supervisor, StartAkkaRpcActor.PropsFactory propsFactory, String endpointId) {
        return Patterns.ask(
                        supervisor,
                        createStartAkkaRpcActorMessage(propsFactory, endpointId),
                        RpcUtils.INF_DURATION)
                .toCompletableFuture()
                .thenApply(SupervisorActor.StartAkkaRpcActorResponse.class::cast)
                .join();
    }
  1. 这边就是类似 ator.ask(msg) 就是向supervisor 发送信息
  2. createStartAkkaRpcActorMessage 就是构建一个 StartAkkaRpcActor Msg 对象
SupervisorActor#createReceive

这个方法其实就是类似scala 中的 case 批判,上层是 akka.actor.AbstractActor#receive 调用这个。不熟悉的需要看下Scala Actor 代码。

    @Override
    public Receive createReceive() {
        return receiveBuilder()
                .match(StartAkkaRpcActor.class, this::createStartAkkaRpcActorMessage)
                .matchAny(this::handleUnknownMessage)
                .build();
    }
  1. 很明了,对应 StartAkkaRpcActor 这种信息,调用 createStartAkkaRpcActorMessage 方法。
SupervisorActor#createStartAkkaRpcActorMessage(SupervisorActor.StartAkkaRpcActor)
    private void createStartAkkaRpcActorMessage(StartAkkaRpcActor startAkkaRpcActor) {
        final String endpointId = startAkkaRpcActor.getEndpointId();
        final AkkaRpcActorRegistration akkaRpcActorRegistration =
                new AkkaRpcActorRegistration(endpointId);

        final Props akkaRpcActorProps =
                startAkkaRpcActor
                        .getPropsFactory()
                        .create(akkaRpcActorRegistration.getInternalTerminationFuture());

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

        try {
            final ActorRef actorRef = getContext().actorOf(akkaRpcActorProps, endpointId);

            registeredAkkaRpcActors.put(actorRef, akkaRpcActorRegistration);

            getSender()
                    .tell(
                            StartAkkaRpcActorResponse.success(
                                    ActorRegistration.create(
                                            actorRef,
                                            akkaRpcActorRegistration
                                                    .getExternalTerminationFuture())),
                            getSelf());
        } catch (AkkaException akkaException) {
            getSender().tell(StartAkkaRpcActorResponse.failure(akkaException), getSelf());
        }
    }
  1. 先构造 AkkaRpcActorRegistration ,这个之后会封装在 Response 中
  2. 通过请求中的PropFactory 获取Rpc 相关的信息。
  3. 拿到当前 ActorSystemImpl 注册上,拿到ActorRef
  4. 最后封装 StartAkkaRpcActorResponse,返回。

AkkaRpcService#registerAkkaRpcActor

AkkaRpcService#startServer

再回到这

  1. 注册 registerAkkaRpcActor
  2. 拿到 actorRegistration,再拿到当前Endpoint的 Actor Ref
  3. 通过 Actor Ref 拿到一些属性
  4. 把当前所有的 RpcGateway 的class 放入Set 中
  5. 构造 InvocationHandler ,也就是 AkkaInvocationHandler,包含 akkaAddress 、hostname 、actorRef ...
  6. 构造动态代理 RpcServer

至此RpcEndpoint 已经准备好了,但是还需要启用。

RpcEndpoint#start

    public final void start() {
        rpcServer.start();
    }
  1. 这个RPCServer 就是 AkkaInvocationHandler

AkkaInvocationHandler#start

public void start() {
    rpcEndpoint.tell(ControlMessages.START, ActorRef.noSender());
}

这个 rpcEndpoint 实际上我们之前注册Endpoint的时候创建的ActorRef,对应是的AkkaRpcActor

AkkaRpcActor#createReceive

   interface State {
        default State start(AkkaRpcActor<?> akkaRpcActor, ClassLoader flinkClassLoader) {
            throw new AkkaRpcInvalidStateException(
                    invalidStateTransitionMessage(StartedState.STARTED));
        }

        default State stop() {
            throw new AkkaRpcInvalidStateException(
                    invalidStateTransitionMessage(StoppedState.STOPPED));
        }

        default State terminate(AkkaRpcActor<?> akkaRpcActor, ClassLoader flinkClassLoader) {
            throw new AkkaRpcInvalidStateException(
                    invalidStateTransitionMessage(TerminatingState.TERMINATING));
        }

        default State finishTermination() {
            return TerminatedState.TERMINATED;
        }

        default boolean isRunning() {
            return false;
        }

        default String invalidStateTransitionMessage(State targetState) {
            return String.format(
                    "AkkaRpcActor is currently in state %s and cannot go into state %s.",
                    this, targetState);
        }
    }
    
    
    @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();
    }
  1. ControlMessages.class 对应 handleControlMessage 方法

AkkaRpcActor#handleControlMessage

实际上这个 AkkaRpcActor 有个状态,初始化的时候 是 this.state = StoppedState.STOPPED; 在处理RPC 请求的时候会进行判断 state,这个状态必须是 STARTED 才可以进行处理。

    private void handleControlMessage(ControlMessages controlMessage) {
        try {
            switch (controlMessage) {
                case START:
                    state = state.start(this, flinkClassLoader);
                    break;
                case STOP:
                    state = state.stop();
                    break;
                case TERMINATE:
                    state = state.terminate(this, flinkClassLoader);
                    break;
                default:
                    handleUnknownControlMessage(controlMessage);
            }
        } catch (Exception e) {
            this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure(e);
            throw e;
        }
    }

现在整个Server 已经准备就绪了。再看Client

Client

public class ClientRPC {
    public static void main(String[] args) {
        ActorSystem defaultActorSystem = AkkaUtils.createDefaultActorSystem();
        RpcService rpcService = new AkkaRpcService(defaultActorSystem, AkkaRpcServiceConfiguration.defaultConfiguration());
        CompletableFuture<HelloGateway> connect = rpcService.connect(args[0], HelloGateway.class);
        try {
            System.out.println(connect.get().hello().get());
        } catch (InterruptedException e) {
            e.printStackTrace();
        } catch (ExecutionException e) {
            e.printStackTrace();
        } finally {
            rpcService.stopService();
        }

    }
}
  1. 和Server一样构造 ActorSystem ,构造 rpcService
  2. 拿到 HelloGateway 是关键。

AkkaRpcService#connect(String, java.lang.Class)

    public <C extends RpcGateway> CompletableFuture<C> connect(
            final String address, final Class<C> clazz) {

        return connectInternal(
                address,
                clazz,
                (ActorRef actorRef) -> {
                    Tuple2<String, String> addressHostname = extractAddressHostname(actorRef);

                    return new AkkaInvocationHandler(
                            addressHostname.f0,
                            addressHostname.f1,
                            actorRef,
                            configuration.getTimeout(),
                            configuration.getMaximumFramesize(),
                            null,
                            captureAskCallstacks,
                            flinkClassLoader);
                });
    }

  1. 这边构造了一个函数 需要传入 actorRef 就可以得到 AkkaInvocationHandler,而在下一步就需要这个 AkkaInvocationHandler
  2. AkkaInvocationHandler 实现了 RpcGateway

AkkaRpcService#connectInternal

    private <C extends RpcGateway> CompletableFuture<C> connectInternal(
            final String address,
            final Class<C> clazz,
            Function<ActorRef, InvocationHandler> invocationHandlerFactory) {
        checkState(!stopped, "RpcService is stopped");

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

        final CompletableFuture<ActorRef> actorRefFuture = resolveActorAddress(address);

        final CompletableFuture<HandshakeSuccessMessage> handshakeFuture =
                actorRefFuture.thenCompose(
                        (ActorRef actorRef) ->
                                AkkaFutureUtils.toJava(
                                        Patterns.ask(
                                                        actorRef,
                                                        new RemoteHandshakeMessage(
                                                                clazz, getVersion()),
                                                        configuration.getTimeout().toMilliseconds())
                                                .<HandshakeSuccessMessage>mapTo(
                                                        ClassTag$.MODULE$
                                                                .<HandshakeSuccessMessage>apply(
                                                                        HandshakeSuccessMessage
                                                                                .class))));

        final CompletableFuture<C> gatewayFuture =
                actorRefFuture.thenCombineAsync(
                        handshakeFuture,
                        (ActorRef actorRef, HandshakeSuccessMessage ignored) -> {
                            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();

                            @SuppressWarnings("unchecked")
                            C proxy =
                                    (C)
                                            Proxy.newProxyInstance(
                                                    classLoader,
                                                    new Class<?>[] {clazz},
                                                    invocationHandler);

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

        return guardCompletionWithContextClassLoader(gatewayFuture, flinkClassLoader);
    }
  1. 获取Server 的 ActorRef
  2. 向 Server的 ActorRef 发送 HandshakeSuccessMessage 消息
  3. 传入 Server的 ActorRef 和 HelloGateway.class,构造一个动态代理
  4. 做一些检验 ,最后封装成 CompletableFuture
  5. 当Client 调用 动态代理的方式时候 就是调用刚刚 AkkaInvocationHandler 的 invoke

AkkaInvocationHandler#invoke

    public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
        Class<?> declaringClass = method.getDeclaringClass();

        Object result;

        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)) {
            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 {
            result = invokeRpc(method, args);
        }

        return result;
    }

  1. 进行判断如果是些本地的 就可以直接调用方法 ,远程的就走 invokeRpc

AkkaInvocationHandler#invokeRpc

    private Object invokeRpc(Method method, Object[] args) throws Exception {
        String methodName = method.getName();
        Class<?>[] parameterTypes = method.getParameterTypes();
        Annotation[][] parameterAnnotations = method.getParameterAnnotations();
        Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);

        final RpcInvocation rpcInvocation =
                createRpcInvocationMessage(
                        method.getDeclaringClass().getSimpleName(),
                        methodName,
                        parameterTypes,
                        args);

        Class<?> returnType = method.getReturnType();

        final Object result;

        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.
            final Throwable callStackCapture = captureAskCallStack ? new Throwable() : null;

            // execute an asynchronous call
            final CompletableFuture<?> resultFuture = ask(rpcInvocation, futureTimeout);

            final CompletableFuture<Object> completableFuture = new CompletableFuture<>();
            resultFuture.whenComplete(
                    (resultValue, failure) -> {
                        if (failure != null) {
                            completableFuture.completeExceptionally(
                                    resolveTimeoutException(
                                            failure, callStackCapture, address, rpcInvocation));
                        } else {
                            completableFuture.complete(
                                    deserializeValueIfNeeded(resultValue, method));
                        }
                    });

            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;
    }

  1. 获取方法的Class ,传入的参数 。根据这些构造一个RpcInvocation 这个就是传输 RPC调用的消息
  2. 获取方法的返回值,判断是否空,如果是空,直接tell 过去
  3. 如果不是空,就是 ask 拿到返回值。进行判断是否需要反序列化
  4. 包装成 completableFuture,判断返回值是否是completableFuture 如果是,就直接返回,否则进行get

AkkaInvocationHandler#createRpcInvocationMessage

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

        if (isLocal) {
            rpcInvocation =
                    new LocalRpcInvocation(declaringClassName, methodName, parameterTypes, args);
        } else {
            try {
                RemoteRpcInvocation remoteRpcInvocation =
                        new RemoteRpcInvocation(
                                declaringClassName, methodName, parameterTypes, args);

                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;
    }
  1. 这边还是会判断是否是local,但是我们是 RemoteRpcInvocation

AkkaRpcActor#createReceive

    @Override
    public Receive createReceive() {
        return ReceiveBuilder.create()
                .match(RemoteHandshakeMessage.class, this::handleHandshakeMessage)
                .match(ControlMessages.class, this::handleControlMessage)
                .matchAny(this::handleMessage)
                .build();
    }
  1. 回到 Server Actor 中, 对应 RemoteRpcInvocation 是 handleMessage 方法

AkkaRpcActor#handleMessage

    private void handleMessage(final Object message) {
        if (state.isRunning()) {
            mainThreadValidator.enterMainThread();

            try {
                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())));
        }
    }
  1. 这边就是判断 state.isRunning() ,这就当时我们进行 start 的时候启用的状态

AkkaRpcActor#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()
                                + '.'));
    }
}
  1. 一看就知道 RemoteRpcInvocation 对应 RpcInvocation

AkkaRpcActor#handleRpcInvocation

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

        try {
            String methodName = rpcInvocation.getMethodName();
            Class<?>[] parameterTypes = rpcInvocation.getParameterTypes();

            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);

                final Method capturedRpcMethod = rpcMethod;
                if (rpcMethod.getReturnType().equals(Void.TYPE)) {
                    // No return value to send back
                    runWithContextClassLoader(
                            () -> capturedRpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()),
                            flinkClassLoader);
                } else {
                    final Object result;
                    try {
                        result =
                                runWithContextClassLoader(
                                        () ->
                                                capturedRpcMethod.invoke(
                                                        rpcEndpoint, rpcInvocation.getArgs()),
                                        flinkClassLoader);
                    } 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();

                    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());
            }
        }
    }
  1. 通过 RpcInvocation 可以拿到 类名和方法名,然后再当前的Actor 的 Endpoint 中寻找方法 ,然后通过反射拿到 Method
  2. 判断Method 的返回值是否为空,如果是空直接调用方法
  3. 如果不是空对结果进行发送

posted on 2022-04-08 15:27  chouc  阅读(207)  评论(0编辑  收藏  举报

导航