[ https://issues.apache.org/jira/browse/FLINK-4383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15420770#comment-15420770 ]
ASF GitHub Bot commented on FLINK-4383: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2365#discussion_r74737963 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java --- @@ -25,33 +25,42 @@ import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; +import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; import org.apache.flink.util.Preconditions; +import org.apache.log4j.Logger; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.io.IOException; import java.lang.annotation.Annotation; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; import java.util.BitSet; import java.util.concurrent.Callable; /** - * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the - * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is + * Invocation handler to be used with an {@link AkkaRpcActor}. The invocation handler wraps the + * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is * executed. */ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor { + private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class); + private final ActorRef rpcServer; // default timeout for asks private final Timeout timeout; - AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout) { + private final long maximumFramesize; + + AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout, long maximumFramesize) { --- End diff -- `isLocalActorRef` should actually be quite cheap since it simply checks whether the host option of the actor's address is defined or not. But, anyway, it makes sense to check it once and store the result. Will add the change. > Check parameters for serializability before sending a remote RpcInvocation > message > ---------------------------------------------------------------------------------- > > Key: FLINK-4383 > URL: https://issues.apache.org/jira/browse/FLINK-4383 > Project: Flink > Issue Type: Sub-task > Components: Distributed Coordination > Reporter: Till Rohrmann > Assignee: Till Rohrmann > > Before sending a remote {{RpcInvocation}} message we should check that the > rpc arguments are serializable. If not we should eagerly fail with an > appropriate exception message. > If we don't do this, then Akka will silently fail serializing the message > without telling the user. -- This message was sent by Atlassian JIRA (v6.3.4#6332)