With Flink 1.6.0 it is no longer needed to specify the number of started
containers (-yn 145). Flink will dynamically allocate containers. That's
also the reason why you don't registered TMs without a running job.
Moreover it it recommended to start every container with a single slot (no
-ys 5). The parallelism should be controlled via the -p option or by the
default parallelism configured in flink-conf.yaml.

The log snippet says that Flink started the TaskManagers. But it seems as
if they could not register at the ResourceManger or could never be started.
Could you check the TM logs to see what they say. If there is nothing
suspicious, then it would be helpful if you could share the complete logs
with us.

Cheers,
Till



On Mon, Sep 17, 2018 at 9:16 AM Subramanya Suresh <ssur...@salesforce.com>
wrote:

> Hi,
> Was suggested here to migrate to 1.6.0 in lieu of Akka/TM lost issues we
> were facing with 1.4.2. I got our Yarn cluster setup and launched our job
> with the command mentioned below
>
> Symptoms:
>
>    - The CLI logs say the Job is submitted but Yarn ResourceManager says
>    only 1 container allocated, that goes up on refresh and then a subsequent
>    refresh shows it back to 1 container allocated.
>    - The UI consistently shows 0 TMs and 0 Slots (see attached).
>    - The exceptions in the UI, shows the below
>    NoResourceAvailalbleException.
>    - Also see below the JobManager logs.
>
> So not sure what gives ? I was able to launch the same job in 1.4.2 and
> immediately get the mentioned TMs and have the job working as it should.
>
>
>
> *Job Submit Parameters:*
> nohup $FLINK_BINARY run \
>     -m yarn-cluster \
>     -c $FLINK_JOB_CLASSNAME \
>     -yst \
>     -ys 5 \
>     -yn 145 \
>     -yjm 20000 \
>     -ytm 20000 \
>     -ynm $YARN_APPLICATION_NAME \
>     -d $FLINK_JOB_JAR \
>             > $FLINK_JOB_LOGS/stdout.log \
>             2> $FLINK_JOB_LOGS/stderr.log \
>             & echo $! > $FLINK_JOB_LOGS/current-run.pid
>
> *Exception:*
>
> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
> Could not allocate all requires slots within timeout of 300000 ms. Slots 
> required: 2, slots allocated: 0
>
>
> *Yarn JobManager Logs:*
>
> 2018-09-17 06:53:18,041 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  - Connecting
> to ResourceManager akka.tcp://
> fl...@hello-world4-30-crz.ops.sfdc.net:41135/user/resourcemanager(9a62f56ce988f5499dbe1d09bd894b8a)
> 2018-09-17 06:53:18,045 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  - Resolved
> ResourceManager address, beginning registration
> 2018-09-17 06:53:18,046 INFO
> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Cannot
> serve slot request, no ResourceManager connected. Adding as pending request
> [SlotRequestId{e1678524024c0d8e7f18b917ad854418}]
> 2018-09-17 06:53:18,046 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  -
> Registration at ResourceManager attempt 1 (timeout=100ms)
> 2018-09-17 06:53:18,048 INFO
> org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  -
> Starting ZooKeeperLeaderRetrievalService
> /leader/31462809fd71ae1c92a11a58dd2f4d24/job_manager_lock.
> 2018-09-17 06:53:18,048 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Registering job manager
> 8a7f0e49aa68e867ef8f058c46414...@akka.tcp://
> fl...@hello-world4-30-crz.ops.sfdc.net:41135/user/jobmanager_0 for job
> 31462809fd71ae1c92a11a58dd2f4d24.
> 2018-09-17 06:53:18,060 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Registered job manager 8a7f0e49aa68e867ef8f058c46414dda
> @akka.tcp://fl...@hello-world4-30-crz.ops.sfdc.net:41135/user/jobmanager_0
> for job 31462809fd71ae1c92a11a58dd2f4d24.
> 2018-09-17 06:53:18,062 INFO
> org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager
> successfully registered at ResourceManager, leader id:
> 9a62f56ce988f5499dbe1d09bd894b8a.
> 2018-09-17 06:53:18,062 INFO
> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Requesting
> new slot [SlotRequestId{e1678524024c0d8e7f18b917ad854418}] and profile
> ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=0,
> nativeMemoryInMB=0, networkMemoryInMB=0} from resource manager.
> 2018-09-17 06:53:18,064 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Request slot with profile ResourceProfile{cpuCores=-1.0,
> heapMemoryInMB=-1, directMemoryInMB=0, nativeMemoryInMB=0,
> networkMemoryInMB=0} for job 31462809fd71ae1c92a11a58dd2f4d24 with
> allocation id AllocationID{8976aac24593aa0d9854fdb569c1d0ac}.
> 2018-09-17 06:53:18,071 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20000, vCores:5>. Number pending requests 1.
> 2018-09-17 06:53:23,191 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000005 - Remaining pending container
> requests: 1
> 2018-09-17 06:53:23,602 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:53:23,603 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:53:34,193 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:53:39,696 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000006 - Remaining pending container
> requests: 1
> 2018-09-17 06:53:40,269 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:53:40,270 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:53:45,703 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:53:51,209 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000007 - Remaining pending container
> requests: 1
> 2018-09-17 06:53:51,365 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:53:51,365 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:53:51,383 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000009 - Remaining pending container
> requests: 0
> 2018-09-17 06:53:51,385 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000009.
> 2018-09-17 06:54:01,714 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:54:07,217 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000011 - Remaining pending container
> requests: 1
> 2018-09-17 06:54:07,263 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:54:07,266 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:54:07,276 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000012 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:07,276 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000012.
> 2018-09-17 06:54:07,276 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000013 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:07,276 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000013.
> 2018-09-17 06:54:12,720 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:54:18,221 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000016 - Remaining pending container
> requests: 1
> 2018-09-17 06:54:18,256 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:54:18,257 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000017 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000017.
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000018 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000018.
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000020 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:18,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000020.
> 2018-09-17 06:54:28,726 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:54:34,229 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000021 - Remaining pending container
> requests: 1
> 2018-09-17 06:54:34,268 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:54:34,269 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000022 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000022.
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000024 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000024.
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000025 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000025.
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000028 - Remaining pending container
> requests: 0
> 2018-09-17 06:54:34,285 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000028.
> 2018-09-17 06:54:39,731 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:54:45,236 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Received new container:
> container_e31_1536964973951_0247_01_000042 - Remaining pending container
> requests: 1
> 2018-09-17 06:54:45,281 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Creating container launch context for TaskManagers
> 2018-09-17 06:54:45,282 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Starting TaskManagers
>
>
>
>
> 2018-09-17 06:58:08,291 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Returning excess container
> container_e31_1536964973951_0247_01_000595.
> 2018-09-17 06:58:13,403 INFO  org.apache.flink.yarn.YarnResourceManager
>                  - Requesting new TaskExecutor container with resources
> <memory:20480, vCores:5>. Number pending requests 1.
> 2018-09-17 06:58:18,045 INFO
> org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Pending
> slot request [SlotRequestId{e1678524024c0d8e7f18b917ad854418}] timed out.
> 2018-09-17 06:58:18,047 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job
> streaming-searches-test (31462809fd71ae1c92a11a58dd2f4d24) switched from
> state RUNNING to FAILING.
> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
> Could not allocate all requires slots within timeout of 300000 ms. Slots
> required: 2, slots allocated: 0
> at
> org.apache.flink.runtime.executiongraph.ExecutionGraph.lambda$scheduleEager$3(ExecutionGraph.java:984)
> at
> java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
> at
> java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$ResultConjunctFuture.handleCompletedFuture(FutureUtils.java:534)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:770)
> at akka.dispatch.OnComplete.internal(Future.scala:258)
> at akka.dispatch.OnComplete.internal(Future.scala:256)
> at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
> at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
> at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
>
> Sincerely,
>
> --
>
> <http://smart.salesforce.com/sig/ssuresh//us_mb/default/link.html>
>

Reply via email to