[ 
https://issues.apache.org/jira/browse/FLINK-4455?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15455431#comment-15455431
 ] 

ASF GitHub Bot commented on FLINK-4455:
---------------------------------------

Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/2449
  
    +1 for moving the `PartitionStateChecker` and 
`ResultPartitionConsumableNotifier` out of the `NetworkEnvironment`.
    
    Few questions and comments:
    
      - Do we need an extra ExecutorService in the TaskManager? I have been 
digging through a bunch of thread dumps over time and there are already many 
threads and pools already. I would really like to avoid having yet another 
Thread pool (creating thread pools should be an extremely careful decision).
    
      The Akka thread pool executor is quite over-provisioned for the few 
actors we actually use. I think it is perfectly feasible to use that one for 
the few extra futures introduced here. In any case, if not reusing the Akka 
executor pool, then the thread pool needs to be shut down in the TaskManager 
runner. Otherwise it creates a leak when running successive local Flink jobs.
    
      - I am a bit consumed about the `SlotEnvironment`. Maybe it is mainly the 
name, but what does it have to do with the slots? Is it not more like a 
network-messages specific *JobManager Connection*?
      
      - The `ResultPartitionConsumableNotifier` could be per `Task` - that way, 
future multi-JobManager assiciations would work seamlessly and it could 
directly call `fail(...)` on the Task without having to go through the 
`TaskManager`. It could probably leave the TaskManager out of the picture 
completely.


> Replace ActorGateways in NetworkEnvironment by interfaces
> ---------------------------------------------------------
>
>                 Key: FLINK-4455
>                 URL: https://issues.apache.org/jira/browse/FLINK-4455
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network, TaskManager
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>
> The {{NetworkEnvironment}} communicates with the outside world 
> ({{TaskManager}} and {{JobManager}}) via {{ActorGateways}}. This bakes in the 
> dependency on actors. 
> In terms of modularization and an improved abstraction (especially wrt 
> Flip-6) I propose to replace the {{ActorGateways}} by interfaces which 
> exposes the required methods. The current implementation would then simply 
> wrap the method calls in messages and send them via the {{ActorGateway}} to 
> the recipient.
> In Flip-6 the {{JobMaster}} and the {{TaskExecutor}} could simply implement 
> these interfaces as part of their RPC contract.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to