Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5931
  
    @sihuazhou and @shuai-xu thank you for your help in understanding the bug 
here.
    
    Let me rephrase it to make sure I understand the problem exactly. The steps 
are the following:
    
      1. JobMaster / SlotPool requests a slot (AllocationID) from the 
ResourceManager
      2. ResourceManager starts a container with a TaskManager
      3. TaskManager registers at ResourceManager, which tells the TaskManager 
to push a slot to the JobManager.
      4. TaskManager container is killed
      5. The ResourceManager does not queue back the slot requests 
(AllocationIDs) that it sent to the previous TaskManager, so the requests are 
lost and need to time out before another attempt is tried.
    
    Some thoughts on how to deal with this:
      - It seems the ResourceManager should put the slots from the TaskManager 
that was failed back to "pending" so they are given to the next TaskManager 
that starts.
      - I assume that is not happening, because there is concern that the 
failure is also detected on the JobManager/SlotPool and retried there and there 
are double re-tries
      - The solution would be to better define the protocol with respect to who 
is responsible for what retries.
    
    Two ideas on how to fix that:
      1. The ResourceManager notifies the SlotPool that a certain set of 
AllocationIDs has failed, and the SlotPool directly retries the allocations, 
resulting in directly starting new containers.
      2. The ResourceManager always retries allocations for AllocationIDs it 
knows. The SlotPool would not retry, it would keep the same allocations always 
unless they are released as unneeded. We would probably need something to make 
sure that the SlotPool can distinguish from different offers of the same 
AllocationID (in case the ResourceManager assumes a timeout but a request goes 
actually through) - possibly something like an attempt-counter (higher wins).
    
    @tillrohrmann also interested in your thoughts here.



---

Reply via email to