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

Hwanju Kim edited comment on FLINK-12260 at 4/26/19 11:30 PM:
--------------------------------------------------------------

I got repro but in somewhat tricky way, since it's definitely rarely happening 
race. But as mentioned, once it falls into this state, it can't get out of the 
state (by assuming that we're not using active resource manager).

In the repro, I injected artificial delay to RM->TM connection on task executor 
registration, which can timeout the first registration request resulting in 2nd 
try. Since RM->TM connection is carried out in a separate thread via akka ask 
call, delaying here can't block the resource manager endpoint mailbox 
processing, so any further request can be processed during the delay. I 
initially added the delay in handling registerTaskExecutorInternal, but as it 
uses RPC's executor, the delay blocks all the further retries, hence not 
reproducing the race. With the delay in TM connection, 2nd task registration 
attempt can overtake the 1st one going ahead with TM registration, and then the 
resumed 1st request unregisters the TM registration. Although I mimicked the 
race on RM side, I think still sender side can also have potential delay (like 
by network) during tell part in akka ask causing timeout and leading to 2nd try 
racing 1st one. The latter was trickier to mimic, so I tried the first 
approach. 

The following is the JM/TM logs.

JM log:
{code:java}
2019-04-26 17:14:44,921 DEBUG org.apache.flink.runtime.rpc.akka.AkkaRpcService 
- Try to connect to remote RPC endpoint with address 
akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0. Returning a 
org.apache.flink.runtime.taskexecutor.TaskExecutorGateway gateway. 
2019-04-26 17:14:44,924 INFO org.apache.flink.runtime.rpc.akka.AkkaRpcService - 
[REPRO] thread 19 attempt 1
2019-04-26 17:14:44,996 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  
            - [REPRO] thread 22 sleep...
2019-04-26 17:14:45,021 DEBUG org.apache.flink.runtime.rpc.akka.AkkaRpcService 
- Try to connect to remote RPC endpoint with address 
akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0. Returning a 
org.apache.flink.runtime.taskexecutor.TaskExecutorGateway gateway.
2019-04-26 17:14:45,022 INFO org.apache.flink.runtime.rpc.akka.AkkaRpcService - 
[REPRO] thread 19 attempt 2
2019-04-26 17:14:45,038 INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - 
Registering TaskManager with ResourceID d0a410ee9060e62e0c7ef9e46f6418da 
(akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0) at ResourceManager 
under instance id fa4408b5412bb8c18a6a7e58fdc8ff18
2019-04-26 17:14:45,093 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Registering 
TaskManager d0a410ee9060e62e0c7ef9e46f6418da under 
fa4408b5412bb8c18a6a7e58fdc8ff18 at the SlotManager.
2019-04-26 17:14:45,997 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  
            - [REPRO] thread 22 done
2019-04-26 17:14:45,998 DEBUG 
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Replacing 
old registration of TaskExecutor d0a410ee9060e62e0c7ef9e46f6418da.
2019-04-26 17:14:45,998 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Unregister 
TaskManager fa4408b5412bb8c18a6a7e58fdc8ff18 from the SlotManager.
2019-04-26 17:14:46,000 INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - 
Registering TaskManager with ResourceID d0a410ee9060e62e0c7ef9e46f6418da 
(akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0) at ResourceManager 
under instance id ebad00b418637d2774b8f131d49cc79e
2019-04-26 17:14:46,000 DEBUG 
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - The 
target with resource ID d0a410ee9060e62e0c7ef9e46f6418da is already been 
monitored.
2019-04-26 17:14:47,387 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Received 
slot report from instance ebad00b418637d2774b8f131d49cc79e.
2019-04-26 17:14:47,387 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Received 
slot report for unknown task manager with instance id 
ebad00b418637d2774b8f131d49cc79e. Ignoring this report.
2019-04-26 17:19:48,045 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job  
(ed0fbfff272391d1f2a98de45fda6453) switched from state RUNNING to FAILING.
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
Could not allocate all requires slots within timeout of 300000 ms. Slots 
required: 1, slots allocated: 0
...
{code}
TM log:

 
{code:java}
2019-04-26 17:14:44,897 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Resolved 
ResourceManager address, beginning registration
2019-04-26 17:14:44,897 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager attempt 1 (timeout=100ms)
2019-04-26 17:14:45,017 DEBUG 
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager 
(akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager) attempt 1 
timed out after 100 ms
2019-04-26 17:14:45,017 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager attempt 2 (timeout=200ms)
2019-04-26 17:14:45,047 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Successful 
registration at resource manager 
akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager under 
registration id fa4408b5412bb8c18a6a7e58fdc8ff18.
2019-04-26 17:14:46,006 DEBUG akka.actor.LocalActorRefProvider(akka://flink)    
            - resolve of path sequence [/temp/$c] failed

{code}
 

 

This is the diff for log and reproduce (against 1.6.2) – 
[^FLINK-12260-repro.diff]

(The current repro method is not systematic rather a tweak due to tricky race)

 

I may be testing with attempt count approach.


was (Author: hwanju):
I got repro but in somewhat tricky way, since it's definitely rarely happening 
race. But as mentioned, once it falls into this state, it can't get out of the 
state (by assuming that we're not using active resource manager).

In the repro, I injected artificial delay to RM->TM connection on task executor 
registration, which can timeout the first registration request resulting in 2nd 
try. Since RM->TM connection is carried out in a separate thread via akka ask 
call, delaying here can't block the resource manager endpoint mailbox 
processing, so any further request can be processed during the delay. I 
initially added the delay in handling registerTaskExecutorInternal, but as it 
uses RPC's executor, the delay blocks all the further retries, hence not 
reproducing the race. With the delay in TM connection, 2nd task registration 
attempt can overtake the 1st one going ahead with TM registration, and then the 
resumed 1st request unregisters the TM registration. Although I mimicked the 
race on RM side, I think still sender side can also have potential delay (like 
by network) during tell part in akka ask causing timeout and leading to 2nd try 
racing 1st one. The latter was trickier to mimic, so I tried the first 
approach. 

The following is the JM/TM logs.

JM log:
{code:java}
2019-04-26 17:14:44,921 DEBUG org.apache.flink.runtime.rpc.akka.AkkaRpcService 
- Try to connect to remote RPC endpoint with address 
akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0. Returning a 
org.apache.flink.runtime.taskexecutor.TaskExecutorGateway gateway. 
2019-04-26 17:14:44,924 INFO org.apache.flink.runtime.rpc.akka.AkkaRpcService - 
[REPRO] thread 19 attempt 1
2019-04-26 17:14:44,996 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  
            - [REPRO] thread 22 sleep...
2019-04-26 17:14:45,021 DEBUG org.apache.flink.runtime.rpc.akka.AkkaRpcService 
- Try to connect to remote RPC endpoint with address 
akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0. Returning a 
org.apache.flink.runtime.taskexecutor.TaskExecutorGateway gateway.
2019-04-26 17:14:45,022 INFO org.apache.flink.runtime.rpc.akka.AkkaRpcService - 
[REPRO] thread 19 attempt 2
2019-04-26 17:14:45,038 INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - 
Registering TaskManager with ResourceID d0a410ee9060e62e0c7ef9e46f6418da 
(akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0) at ResourceManager 
under instance id fa4408b5412bb8c18a6a7e58fdc8ff18
2019-04-26 17:14:45,093 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Registering 
TaskManager d0a410ee9060e62e0c7ef9e46f6418da under 
fa4408b5412bb8c18a6a7e58fdc8ff18 at the SlotManager.
2019-04-26 17:14:45,997 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  
            - [REPRO] thread 22 done
2019-04-26 17:14:45,998 DEBUG 
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Replacing 
old registration of TaskExecutor d0a410ee9060e62e0c7ef9e46f6418da.
2019-04-26 17:14:45,998 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Unregister 
TaskManager fa4408b5412bb8c18a6a7e58fdc8ff18 from the SlotManager.
2019-04-26 17:14:46,000 INFO  
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - 
Registering TaskManager with ResourceID d0a410ee9060e62e0c7ef9e46f6418da 
(akka.ssl.tcp://flink@192.168.69.15:6122/user/taskmanager_0) at ResourceManager 
under instance id ebad00b418637d2774b8f131d49cc79e
2019-04-26 17:14:46,000 DEBUG 
org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - The 
target with resource ID d0a410ee9060e62e0c7ef9e46f6418da is already been 
monitored.
2019-04-26 17:14:47,387 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Received 
slot report from instance ebad00b418637d2774b8f131d49cc79e.
2019-04-26 17:14:47,387 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager  - Received 
slot report for unknown task manager with instance id 
ebad00b418637d2774b8f131d49cc79e. Ignoring this report.
2019-04-26 17:19:48,045 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job  
(ed0fbfff272391d1f2a98de45fda6453) switched from state RUNNING to FAILING.
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
Could not allocate all requires slots within timeout of 300000 ms. Slots 
required: 1, slots allocated: 0
...
{code}
TM log:

 
{code:java}
2019-04-26 17:14:44,897 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Resolved 
ResourceManager address, beginning registration
2019-04-26 17:14:44,897 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager attempt 1 (timeout=100ms)
2019-04-26 17:14:45,017 DEBUG 
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager 
(akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager) attempt 1 
timed out after 100 ms
2019-04-26 17:14:45,017 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at 
ResourceManager attempt 2 (timeout=200ms)
2019-04-26 17:14:45,047 INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor            - Successful 
registration at resource manager 
akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager under 
registration id fa4408b5412bb8c18a6a7e58fdc8ff18.
2019-04-26 17:14:46,006 DEBUG akka.actor.LocalActorRefProvider(akka://flink)    
            - resolve of path sequence [/temp/$c] failed

{code}
 

 

This is the diff for log and reproduce (against 1.6.2) – 
[^FLINK-12260-repro.diff]

(The current repro method is not systematic rather a tweak due to tricky race)

 

> Slot allocation failure by taskmanager registration timeout and race
> --------------------------------------------------------------------
>
>                 Key: FLINK-12260
>                 URL: https://issues.apache.org/jira/browse/FLINK-12260
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.6.3
>            Reporter: Hwanju Kim
>            Priority: Critical
>         Attachments: FLINK-12260-repro.diff
>
>
>  
> In 1.6.2., we have seen slot allocation failure keep happening for long time. 
> Having looked at the log, I see the following behavior:
>  # TM sends a registration request R1 to resource manager.
>  # R1 times out after 100ms, which is initial timeout.
>  # TM retries a registration request R2 to resource manager (with timeout 
> 200ms).
>  # R2 arrives first at resource manager and registered, and then TM gets 
> successful response moving onto step 5 below.
>  # On successful registration, R2's instance is put to 
> taskManagerRegistrations
>  # Then R1 arrives at resource manager and realizes the same TM resource ID 
> is already registered, which then unregisters R2's instance ID from 
> taskManagerRegistrations. A new instance ID for R1 is registered to 
> workerRegistration.
>  # R1's response is not handled though since it already timed out (see akka 
> temp actor resolve failure below), hence no registration to 
> taskManagerRegistrations.
>  # TM keeps heartbeating to the resource manager with slot status.
>  # Resource manager ignores this slot status, since taskManagerRegistrations 
> contains R2, not R1, which replaced R2 in workerRegistration at step 6.
>  # Slot request can never be fulfilled, timing out.
> The following is the debug logs for the above steps:
>  
> {code:java}
> JM log:
> 2019-04-11 22:39:40.000,Registering TaskManager with ResourceID 
> 46c8e0d0fcf2c306f11954a1040d5677 
> (akka.ssl.tcp://flink@flink-taskmanager:6122/user/taskmanager_0) at 
> ResourceManager
> 2019-04-11 22:39:40.000,Registering TaskManager 
> 46c8e0d0fcf2c306f11954a1040d5677 under deade132e2c41c52019cdc27977266cf at 
> the SlotManager.
> 2019-04-11 22:39:40.000,Replacing old registration of TaskExecutor 
> 46c8e0d0fcf2c306f11954a1040d5677.
> 2019-04-11 22:39:40.000,Unregister TaskManager 
> deade132e2c41c52019cdc27977266cf from the SlotManager.
> 2019-04-11 22:39:40.000,Registering TaskManager with ResourceID 
> 46c8e0d0fcf2c306f11954a1040d5677 
> (akka.ssl.tcp://flink@flink-taskmanager:6122/user/taskmanager_0) at 
> ResourceManager
> TM log:
> 2019-04-11 22:39:40.000,Registration at ResourceManager attempt 1 
> (timeout=100ms)
> 2019-04-11 22:39:40.000,Registration at ResourceManager 
> (akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager) attempt 1 
> timed out after 100 ms
> 2019-04-11 22:39:40.000,Registration at ResourceManager attempt 2 
> (timeout=200ms)
> 2019-04-11 22:39:40.000,Successful registration at resource manager 
> akka.ssl.tcp://flink@flink-jobmanager:6123/user/resourcemanager under 
> registration id deade132e2c41c52019cdc27977266cf.
> 2019-04-11 22:39:41.000,resolve of path sequence [/temp/$c] failed{code}
>  
> As RPC calls seem to use akka ask, which creates temporary source actor, I 
> think multiple RPC calls could've arrived out or order by different actor 
> pairs and the symptom above seems to be due to that. If so, it could have 
> attempt account in the call argument to prevent unexpected unregistration? At 
> this point, what I have done is only log analysis, so I could do further 
> analysis, but before that wanted to check if it's a known issue. I also 
> searched with some relevant terms and log pieces, but couldn't find the 
> duplicate. Please deduplicate if any.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to