Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-20005

"Kerberized YARN application" test unstable

    XMLWordPrintableJSON

Details

    Description

      https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=9066&view=logs&j=c88eea3b-64a0-564d-0031-9fdcd7b8abee&t=ff888d9b-cd34-53cc-d90f-3e446d355529

      The Running Kerberized YARN application on Docker test (default input) is failing.

      These are some exceptions spotted in the logs:

      2020-11-05T14:22:29.3315695Z Nov 05 14:22:29 2020-11-05 14:21:52,696 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Flat Map (2/3) (7806b7a7074425c5ff0906befd94e122) switched from SCHEDULED to FAILED on not deployed.
      2020-11-05T14:22:29.3318307Z Nov 05 14:22:29 java.util.concurrent.CompletionException: org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Slot request bulk is not fulfillable! Could not allocate the required slot within slot request timeout
      2020-11-05T14:22:29.3320512Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3322173Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3323809Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:607) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3325448Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3331094Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3332769Z Nov 05 14:22:29 	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3335736Z Nov 05 14:22:29 	at org.apache.flink.runtime.scheduler.SharedSlot.cancelLogicalSlotRequest(SharedSlot.java:195) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3342621Z Nov 05 14:22:29 	at org.apache.flink.runtime.scheduler.SlotSharingExecutionSlotAllocator.cancelLogicalSlotRequest(SlotSharingExecutionSlotAllocator.java:147) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3348463Z Nov 05 14:22:29 	at org.apache.flink.runtime.scheduler.SharingPhysicalSlotRequestBulk.cancel(SharingPhysicalSlotRequestBulk.java:84) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3353749Z Nov 05 14:22:29 	at org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkWithTimestamp.cancel(PhysicalSlotRequestBulkWithTimestamp.java:66) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3362495Z Nov 05 14:22:29 	at org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.lambda$schedulePendingRequestBulkWithTimestampCheck$0(PhysicalSlotRequestBulkCheckerImpl.java:87) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3366937Z Nov 05 14:22:29 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3370686Z Nov 05 14:22:29 	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_272]
      2020-11-05T14:22:29.3380715Z Nov 05 14:22:29 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:404) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3384436Z Nov 05 14:22:29 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:197) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3387431Z Nov 05 14:22:29 	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3390333Z Nov 05 14:22:29 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:154) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3392937Z Nov 05 14:22:29 	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3395430Z Nov 05 14:22:29 	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3397949Z Nov 05 14:22:29 	at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3401799Z Nov 05 14:22:29 	at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3449637Z Nov 05 14:22:29 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3452289Z Nov 05 14:22:29 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3454833Z Nov 05 14:22:29 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3458801Z Nov 05 14:22:29 	at akka.actor.Actor$class.aroundReceive(Actor.scala:517) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3469564Z Nov 05 14:22:29 	at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3472736Z Nov 05 14:22:29 	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3475094Z Nov 05 14:22:29 	at akka.actor.ActorCell.invoke(ActorCell.scala:561) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3478753Z Nov 05 14:22:29 	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3497848Z Nov 05 14:22:29 	at akka.dispatch.Mailbox.run(Mailbox.scala:225) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3516200Z Nov 05 14:22:29 	at akka.dispatch.Mailbox.exec(Mailbox.scala:235) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3519594Z Nov 05 14:22:29 	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3522331Z Nov 05 14:22:29 	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3524990Z Nov 05 14:22:29 	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3528102Z Nov 05 14:22:29 	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3530334Z Nov 05 14:22:29 Caused by: org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Slot request bulk is not fulfillable! Could not allocate the required slot within slot request timeout
      2020-11-05T14:22:29.3534080Z Nov 05 14:22:29 	at org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.lambda$schedulePendingRequestBulkWithTimestampCheck$0(PhysicalSlotRequestBulkCheckerImpl.java:84) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3536451Z Nov 05 14:22:29 	... 24 more
      2020-11-05T14:22:29.3537535Z Nov 05 14:22:29 Caused by: java.util.concurrent.TimeoutException: Timeout has occurred: 120000 ms
      2020-11-05T14:22:29.3540969Z Nov 05 14:22:29 	at org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.lambda$schedulePendingRequestBulkWithTimestampCheck$0(PhysicalSlotRequestBulkCheckerImpl.java:84) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
      2020-11-05T14:22:29.3542868Z Nov 05 14:22:29 	... 24 more
      
      2020-11-05T14:22:14.3964651Z Nov 05 14:22:13 20/11/05 14:21:55 INFO rmapp.RMAppImpl: application_1604585664395_0001 State change from RUNNING to FINAL_SAVING on event=ATTEMPT_FAILED
      2020-11-05T14:22:14.3966539Z Nov 05 14:22:13 20/11/05 14:21:55 INFO recovery.RMStateStore: Updating info for app: application_1604585664395_0001
      2020-11-05T14:22:14.3968255Z Nov 05 14:22:13 20/11/05 14:21:55 INFO capacity.CapacityScheduler: Application Attempt appattempt_1604585664395_0001_000001 is done. finalState=FAILED
      2020-11-05T14:22:14.3970618Z Nov 05 14:22:13 20/11/05 14:21:55 INFO rmapp.RMAppImpl: Application application_1604585664395_0001 failed 1 times (global limit =2; local limit is =1) due to AM Container for appattempt_1604585664395_0001_000001 exited with  exitCode: 2
      2020-11-05T14:22:14.3973331Z Nov 05 14:22:13 Failing this attempt.Diagnostics: Exception from container-launch.
      2020-11-05T14:22:14.3974475Z Nov 05 14:22:13 Container id: container_1604585664395_0001_01_000001
      2020-11-05T14:22:14.3975384Z Nov 05 14:22:13 Exit code: 2
      2020-11-05T14:22:14.3976946Z Nov 05 14:22:13 Stack trace: org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException: Launch container failed
      2020-11-05T14:22:14.3979115Z Nov 05 14:22:13 	at org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DefaultLinuxContainerRuntime.launchContainer(DefaultLinuxContainerRuntime.java:112)
      2020-11-05T14:22:14.3981642Z Nov 05 14:22:13 	at org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DelegatingLinuxContainerRuntime.launchContainer(DelegatingLinuxContainerRuntime.java:130)
      2020-11-05T14:22:14.3983756Z Nov 05 14:22:13 	at org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor.launchContainer(LinuxContainerExecutor.java:395)
      2020-11-05T14:22:14.3985627Z Nov 05 14:22:13 	at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:299)
      2020-11-05T14:22:14.3987444Z Nov 05 14:22:13 	at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:83)
      2020-11-05T14:22:14.3989017Z Nov 05 14:22:13 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      2020-11-05T14:22:14.3990393Z Nov 05 14:22:13 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      2020-11-05T14:22:14.3991866Z Nov 05 14:22:13 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      2020-11-05T14:22:14.3993133Z Nov 05 14:22:13 	at java.lang.Thread.run(Thread.java:748)
      2020-11-05T14:22:14.3993947Z Nov 05 14:22:13 
      2020-11-05T14:22:14.3994706Z Nov 05 14:22:13 Shell output: main : command provided 1
      

      Attachments

        Activity

          People

            rmetzger Robert Metzger
            rmetzger Robert Metzger
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: