Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2381#discussion_r75140948
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
 ---
    @@ -197,11 +201,11 @@ public void stopService() {
        }
     
        @Override
    -   public String getAddress(RpcGateway selfGateway) {
    +   public String getAddress(RpcGateway gateway) {
                checkState(!stopped, "RpcService is stopped");
     
    -           if (selfGateway instanceof AkkaGateway) {
    -                   ActorRef actorRef = ((AkkaGateway) 
selfGateway).getRpcEndpoint();
    +           if (gateway instanceof AkkaGateway) {
    +                   ActorRef actorRef = ((AkkaGateway) 
gateway).getActorRef();
                        return AkkaUtils.getAkkaURL(actorSystem, actorRef);
    --- End diff --
    
    This will be problematic if you pass in a local actor ref which has been 
created by a different actor system. Because then you will return a wrong 
remote address.
    
    So for example, the following test would fail:
    ```
    @Test
        public void test() {
                LinkedBlockingQueue<Object> linkedBlockingQueue = new 
LinkedBlockingQueue<>();
                TestEndpoint testEndpoint = new 
TestEndpoint(akkaRpcService1,linkedBlockingQueue);
                testEndpoint.start();
    
                TestEndpoint testEndpoint2 = new TestEndpoint(akkaRpcService2, 
linkedBlockingQueue);
                testEndpoint2.start();
    
                assertEquals(testEndpoint2.getAddress(), 
akkaRpcService1.getAddress(testEndpoint2.getSelf()));
                assertEquals(testEndpoint.getAddress(), 
akkaRpcService2.getAddress(testEndpoint.getSelf()));
        }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to