You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Elias Levy (JIRA)" <ji...@apache.org> on 2018/10/30 15:28:00 UTC

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

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

Elias Levy updated FLINK-10617:
-------------------------------
    Priority: Critical  (was: Major)

> 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, 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 AllocationID{740caf20a5f7f767864122dc9a7444d9}.
> 21:36:49,688 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,688 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,688 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,689 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request AllocationID{2ca95a9b9ccbd23d235b338d9aff7e56} for job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,689 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{2ca95a9b9ccbd23d235b338d9aff7e56}.
> 21:36:49,689 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,689 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,689 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,694 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request AllocationID{0521fab5d106362671db3b18031685a7} for job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,694 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{0521fab5d106362671db3b18031685a7}.
> 21:36:49,694 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,694 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,694 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,695 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,696 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request AllocationID{f88e958c2c13a27f6ebaca68892c6554} for job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,696 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{f88e958c2c13a27f6ebaca68892c6554}.
> 21:36:49,696 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,696 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,696 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,698 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request AllocationID{229f7519d895335cff7b577364d3f034} for job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,698 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{229f7519d895335cff7b577364d3f034}.
> 21:36:49,698 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,698 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,698 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,699 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Resolved JobManager address, beginning registration
> 21:36:49,699 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request AllocationID{98341da2fd62db5e0a775dd9196a522e} for job 87c61e8ee64cdbd50f191d39610eb58f from resource manager with leader id 8e06aa64d5f8961809da38fe7f224cc1.
> 21:36:49,700 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{98341da2fd62db5e0a775dd9196a522e}.
> 21:36:49,700 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job 87c61e8ee64cdbd50f191d39610eb58f for job leader monitoring.
> 21:36:49,700 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,700 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/87c61e8ee64cdbd50f191d39610eb58f/job_manager_lock.
> 21:36:49,703 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,706 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Resolved JobManager address, beginning registration
> 21:36:49,706 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Registration at JobManager attempt 1 (timeout=100ms)
> 21:36:49,708 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Successful registration at job manager akka.tcp://flink@172.31.18.172:6123/user/jobmanager_3 for job 87c61e8ee64cdbd50f191d39610eb58f.
> 21:36:49,709 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Establish JobManager connection for job 87c61e8ee64cdbd50f191d39610eb58f.
> 21:36:49,712 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Offer reserved slots to the leader of job 87c61e8ee64cdbd50f191d39610eb58f.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{229f7519d895335cff7b577364d3f034}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{98341da2fd62db5e0a775dd9196a522e}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{f88e958c2c13a27f6ebaca68892c6554}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{0521fab5d106362671db3b18031685a7}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{4274d96a59d370305520876f5b84fb9f}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{3a64e2c8c5b22adbcfd3ffcd2b49e7f9}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{2ca95a9b9ccbd23d235b338d9aff7e56}.
> 21:36:49,713 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Activate slot AllocationID{740caf20a5f7f767864122dc9a7444d9}.
> {noformat}
> Alas, the job failed to restore, timing out.  JM logs show it requests 8 slots, but that some timeout, causing the restore to fail:
> {noformat}
> 21:36:49,716 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{4e63f5ba519d83764a2e06611285d930}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,716 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{7fa61d06d579e3ac55456b46e7f6333e}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,716 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{f5b48cd43ba142fe90f73acc7e69ae76}.
> 21:36:49,716 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{a12f5de011daeb570b9afacf7d3241ab}.
> 21:36:49,716 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{a7197d88984291a7b89beda98ae351d4}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,717 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{352b4ea7d7bfe4f4910f5c40c96d1684}.
> 21:36:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{c88a96628d5a13e5ee14371f62f45866}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{e81eb2b9ef6bf9b5a6f2299b69328b80}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{dfa16526ccec6297ba6587d9fbd60993}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{10ad63aceecbb72a709f57b3a6f13437}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,719 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting new slot [SlotRequestId{8462b3a4890330f261ab41208e863d00}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 21:36:49,717 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{eff7d3b200a6c225fb3c49ab5d5fc5b4}.
> 21:36:49,719 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{54dea082f27a4c6848fd539292c78e83}.
> 21:36:49,719 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{e69fb15ef711f56b7582f8e507f30af2}.
> 21:36:49,719 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{8d2711dd73157f929263e08db873334f}.
> 21:36:49,719 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0, networkMemoryInMB=0} for job 87c61e8ee64cdbd50f191d39610eb58f with allocation id AllocationID{2480907777440beb3accbb559b060a3c}.
> 21:41:49,716 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{4e63f5ba519d83764a2e06611285d930}] timed out.
> 21:41:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{7fa61d06d579e3ac55456b46e7f6333e}] timed out.
> 21:41:49,717 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{a7197d88984291a7b89beda98ae351d4}] timed out.
> 21:41:49,719 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Foo (87c61e8ee64cdbd50f191d39610eb58f) switched from state RUNNING to FAILING.
> Could not allocate all requires slots within timeout of 300000 ms. Slots required: 384, slots allocated: 369
> org.apache.flink.runtime.executiongraph.ExecutionGraph.lambda$scheduleEager$3(ExecutionGraph.java:984)
> java.util.concurrent.CompletableFuture.uniExceptionally(Unknown Source)
> java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(Unknown Source)
> java.util.concurrent.CompletableFuture.postComplete(Unknown Source)
> java.util.concurrent.CompletableFuture.completeExceptionally(Unknown Source)
> org.apache.flink.runtime.concurrent.FutureUtils$ResultConjunctFuture.handleCompletedFuture(FutureUtils.java:534)
> java.util.concurrent.CompletableFuture.uniWhenComplete(Unknown Source)
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(Unknown Source)
> java.util.concurrent.CompletableFuture.postComplete(Unknown Source)
> java.util.concurrent.CompletableFuture.completeExceptionally(Unknown Source)
> org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:770)
> akka.dispatch.OnComplete.internal(Future.scala:258)
> akka.dispatch.OnComplete.internal(Future.scala:256)
> akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
> akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
> scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
> org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
> akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
> akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:20)
> akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:18)
> scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:436)
> scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:435)
> scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
> akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
> akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
> akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> 21:41:49,726 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{c88a96628d5a13e5ee14371f62f45866}] timed out.
> 21:41:49,726 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{e81eb2b9ef6bf9b5a6f2299b69328b80}] timed out.
> 21:41:49,726 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{dfa16526ccec6297ba6587d9fbd60993}] timed out.
> 21:41:49,726 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{10ad63aceecbb72a709f57b3a6f13437}] timed out.
> 21:41:49,726 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending slot request [SlotRequestId{8462b3a4890330f261ab41208e863d00}] timed out.
> {noformat}
>  
> This repeats itself until the job is canceled.
>  



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