[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2020-03-03 Thread Till Rohrmann (Jira)


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

Till Rohrmann commented on FLINK-10617:
---

[~sharonxr55] were you able to reproduce the problem?

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.6.1, 1.6.2
>Reporter: Elias Levy
>Priority: Critical
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,683 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{740caf20a5f7f767864122dc9a7444d9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Registration at JobManager attempt 1 (timeout=100ms)
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for 

[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2019-10-21 Thread Till Rohrmann (Jira)


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

Till Rohrmann commented on FLINK-10617:
---

Ok, then let me know once you managed to make the problem reproducible in order 
to debug it further.

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.6.1, 1.6.2
>Reporter: Elias Levy
>Priority: Critical
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,683 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{740caf20a5f7f767864122dc9a7444d9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Registration at JobManager attempt 1 (timeout=100ms)
> 21:36:49,688 INFO  

[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2019-10-20 Thread Sharon Xie (Jira)


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

Sharon Xie commented on FLINK-10617:


We can’t even reproduce it with the current system (1.5.5). The guess is 
something weird happened to the  JM which put it in an unexpected state. 
Because of this, used slots can’t be released to the slot pool. 

The other piece We know is that the new replica set of TMs cleared the weird 
state and made all used slots released somehow.

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.6.1, 1.6.2
>Reporter: Elias Levy
>Priority: Critical
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,683 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{740caf20a5f7f767864122dc9a7444d9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 

[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2019-10-17 Thread Till Rohrmann (Jira)


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

Till Rohrmann commented on FLINK-10617:
---

Have you tried whether the same problem still occurs with one of Flink's latest 
versions?

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.6.1, 1.6.2
>Reporter: Elias Levy
>Priority: Critical
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,683 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{740caf20a5f7f767864122dc9a7444d9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Registration at JobManager attempt 1 (timeout=100ms)
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor  

[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2019-10-16 Thread Sharon Xie (Jira)


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

Sharon Xie commented on FLINK-10617:


I'm seeing the similar  issue with Flink 1.5.5.

 

We had 90 TMs with 6 task slots on each TM deployed in a K8S cluster. Under 
normal situation, ~480 slots are used running about 90 jobs

The symptom is:
 * JM restarted and all the jobs are being recovered
 * JM reported no slots available and we found that no slots were released when 
the jobs were restarted.
 * Looking at the log, a few jobs reported logs like: 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
Could not allocate all requires slots within timeout of 30 ms. Slots 
required: 100, slots allocated: 73 (the same job reported the same number of 
slots allocated but was never able to move beyond that). This looked like the 
system stuck in a live lock.
 * We increased the TM replica count to 115 and there were 690 total task 
slots. After this, a few jobs were recovered because of the change. However, we 
still didn't see any used slots being released. There were still a few jobs 
stuck in the live lock.
 * We edited the TM deployment, which essentially created a new replica set for 
TMs and then the issue was resolved. The number of current used slots returned 
to ~480.

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.6.1, 1.6.2
>Reporter: Elias Levy
>Priority: Critical
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting 

[jira] [Commented] (FLINK-10617) Restoring job fails because of slot allocation timeout

2018-10-30 Thread Elias Levy (JIRA)


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

Elias Levy commented on FLINK-10617:


I've retested with 1.6.2 and have confirmed the issue still exists, even though 
FLINK-9932 is fixed.

> Restoring job fails because of slot allocation timeout
> --
>
> Key: FLINK-10617
> URL: https://issues.apache.org/jira/browse/FLINK-10617
> Project: Flink
>  Issue Type: Bug
>  Components: ResourceManager, TaskManager
>Affects Versions: 1.6.1
>Reporter: Elias Levy
>Priority: Major
>
> The following may be related to FLINK-9932, but I am unsure.  If you believe 
> it is, go ahead and close this issue and a duplicate.
> While trying to test local state recovery on a job with large state, the job 
> failed to be restored because slot allocation timed out.
> The job is running on a standalone cluster with 12 nodes and 96 task slots (8 
> per node).  The job has parallelism of 96, so it consumes all of the slots, 
> and has ~200 GB of state in RocksDB.  
> To test local state recovery I decided to kill one of the TMs.  The TM 
> immediately restarted and re-registered with the JM.  I confirmed the JM 
> showed 96 registered task slots.
> {noformat}
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Resolved ResourceManager address, beginning registration
> 21:35:44,616 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Registration at ResourceManager attempt 1 (timeout=100ms)
> 21:35:44,640 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Successful registration at resource manager 
> akka.tcp://flink@172.31.18.172:6123/user/resourcemanager under registration 
> id 302988dea6afbd613bb2f96429b65d18.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{4274d96a59d370305520876f5b84fb9f} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,667 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,668 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,671 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Allocated slot for AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,681 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Stopping ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,681 INFO  
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - 
> Starting ZooKeeperLeaderRetrievalService 
> /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,683 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Try to register at job manager 
> akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 with leader id 
> f85f6f9b-7713-4be3-a8f0-8443d91e5e6d.
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Resolved JobManager address, beginning registration
> 21:36:49,687 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor 
>- Receive slot request AllocationID{740caf20a5f7f767864122dc9a7444d9} for 
> job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 
> 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService 
>- Registration at JobManager attempt 1 (timeout=100ms)
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor