[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-05-03 Thread Hwanju Kim (JIRA)


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

Hwanju Kim commented on FLINK-12260:


Sure. I will ping you once I will have tested the fix.

> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-05-02 Thread Till Rohrmann (JIRA)


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

Till Rohrmann commented on FLINK-12260:
---

Thanks a lot for your detailed analysis of the ask code path in Akka [~hwanju]. 
Do you wanna open a PR for this fix? You can ping me for the review.

> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-30 Thread Hwanju Kim (JIRA)


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

Hwanju Kim commented on FLINK-12260:


Thanks for the clarification. I thought you meant it without introducing any 
additional map, but now it seems clear.

I had tried thinking conservative approach as I couldn't 100% rule out the 
possibility of sender-side race. As we may have a potential simpler solution, I 
looked at the code again a little further. Initially what led me to any 
possibility of race is this part:

 
{code:java}
val a = PromiseActorRef(ref.provider, timeout, targetName = actorRef, 
message.getClass.getName, sender)
actorRef.tell(message, a)
a.result.future
{code}
This is internalAsk from invokeRpc and PromiseActorRef internally does 
scheduler.scheduleOnce(timeout.duration) for the timer. The tell of actorRef is 
sending a message to RM through RemoteActorRef and EndpointManager where the 
message is passed to Dispatcher, which enqueues the message to mbox and 
executes mbox via executor thread. My impression was that as tell is 
asynchronous via executor service, the timer of PromiseActorRef set up before 
can fire before the message hit the road off the sender. Although that'd be 
possible, the message at least seems to be enqueued to mbox for RM endpoint and 
thus the order can be preserved against the next attempt after timeout. So, the 
ordering seems fine. In addition I was also concerned the case where two 
different ask calls might happen to use two different TCP connections leading 
any possible out-of-order delivery. Although not 100% exercising the relevant 
code, it seems to use a single connection associated by akka endpoints and I 
checked that's true by packet capture. 

So, based on the code inspection and no successful repro on sender-side, we can 
currently conclude that the race is likely happening in task executor 
connection/handshake on the receiver-side (as repro does). I will test it out 
with the Till's proposal. On our side, once this fix ends up being applied, we 
can keep eyes on our test apps, which intermittently hit this issue, to see if 
there's any other race issue.

 

 

> 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 
> 

[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-30 Thread Till Rohrmann (JIRA)


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

Till Rohrmann commented on FLINK-12260:
---

I'm still a bit skeptical that there is a race condition on the sender side. We 
only trigger the second registration call after the first one has produced a 
timeout. A timeout can only be produced if we have sent a request to the 
receiver side.

My proposal would be the following:

We have a {{Map> 
taskExecutorGatewayFutures}} of pending {{TaskExecutor}} connections. When we 
enter the method {{ResourceManager#registerTaskExecutor}} we create a new 
future and update {{taskExecutorGatewayFutures}}.

Inside the {{handleAsync}} call on the future we check whether we are still the 
pending {{TaskExecutor}} connection:

{code}
return taskExecutorGatewayFuture.handleAsync(
(TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> {
if (taskExecutorGatewayFutures.get(taskExecutorResourceId) == 
taskExecutorGatewayFuture) {

taskExecutorGatewayFutures.remove(taskExecutorResourceId);
if (throwable != null) {
return new 
RegistrationResponse.Decline(throwable.getMessage());
} else {
return registerTaskExecutorInternal(
taskExecutorGateway,
taskExecutorAddress,
taskExecutorResourceId,
dataPort,
hardwareDescription);
}
} else {
log.debug("Ignoring outdated TaskExecutorGateway 
connection.");
}
},
getMainThreadExecutor());
{code}

Given that the {{registerTaskExecutor}} calls come in order, this should 
prevent that an earlier register call overrides a later one.

> 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 
> 

[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-30 Thread Hwanju Kim (JIRA)


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

Hwanju Kim commented on FLINK-12260:


Thanks Till. I had tested with attempt count addition to RPC before your 
comment. I worked as expected.

At any rate, I may need some clarification on your idea about reference 
equality, regarding how it could address this issue. Although the repro uses 
delay in task executor connection, as mentioned, the race may happen on sender 
side. Even for dealing with race in the task executor connection, I am curious 
how reference check alone would work. If receiver-only approach can work, it 
would be better compared to attempt count one.

> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-29 Thread Till Rohrmann (JIRA)


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

Till Rohrmann commented on FLINK-12260:
---

[~jiaxl] I think [~hwanju] is already working on a fix for it.

[~hwanju] if I understood you correctly, then I think the solution could be 
keeping a reference to the {{CompletableFuture 
taskExecutorGatewayFuture}} future returned in {{ResourceManager.java:373}} and 
then check in the callback for referential equality 
{{ResourceManager.java:376}}. Then we would not have to add another field to 
the RPC call.

This is a really good discovery. Thanks a lot for investigating this problem!

> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-28 Thread Xiaolei Jia (JIRA)


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

Xiaolei Jia commented on FLINK-12260:
-

Hi,

If this issue is still open, I'd like to fix this problem with an attempt 
count, as [~till.rohrmann] suggested.  

Thanks.

> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-26 Thread Hwanju Kim (JIRA)


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

Hwanju Kim commented on FLINK-12260:


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 30 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 

[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-23 Thread Hwanju Kim (JIRA)


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

Hwanju Kim commented on FLINK-12260:


Thanks Till for the comment. I haven't attempted to reproduce the issue 
locally, and I will try reproducing the issue with the better logging and will 
update this thread once I get more info.

Regarding odd timestamps of the logs, this is our logging issue we plan to fix, 
which loses millisecond granularity when sending flink logs to long-term log 
archive. As this was the postmortem analysis, I couldn't gather the live flink 
logs, that was why the logs zeroed out milliseconds part.

> 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: Major
>
>  
> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-23 Thread Till Rohrmann (JIRA)


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

Till Rohrmann commented on FLINK-12260:
---

I think introducing an attempt counter should solve the problem. But before we 
fix it that way I would like to fully understand how this out of orderness 
happens.

> 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: Major
>
>  
> 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)


[jira] [Commented] (FLINK-12260) Slot allocation failure by taskmanager registration timeout and race

2019-04-23 Thread Till Rohrmann (JIRA)


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

Till Rohrmann commented on FLINK-12260:
---

Thanks for reporting this issue [~hwanju]. This sounds like a Flink problem to 
me. Are you able to reproduce the problem? If yes, then it would be helpful to 
modify the logging statement in line {{ResourceManager.java:727}} into 
{{log.info("Registering TaskManager with ResourceID {} ({}) at ResourceManager 
under instance id {}", taskExecutorResourceId, taskExecutorAddress, 
registration.getInstanceID());}}.

Something which looks a bit odd is that all logging statement happened at the 
same time even though there should be a timeout of {{100}} ms in between. 

At the moment I'm not sure whether I can fully backtrack the problem to its 
cause. Maybe the problem is caused by the {{RetryingRegistration}} which uses a 
thread pool of multiple threads to send the registration requests. The 
additional logs would be tremendously helpful for further debugging this 
problem.

> 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: Major
>
>  
> 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)