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

Job failed when enable pipelined-shuffle.compression and numberOfTaskSlots > 1

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Blocker
    • Resolution: Fixed
    • 1.10.0
    • 1.10.0
    • Runtime / Network
    • $ git log
      commit 4b54da2c67692b1c9d43e1184c00899b0151b3ae
      Author: bowen.li <bowenli86@gmail.com>
      Date: Tue Dec 17 17:37:03 2019 -0800

    Description

      Job worked well with default flink-conf.yaml with pipelined-shuffle.compression:

      taskmanager.numberOfTaskSlots: 1
      taskmanager.network.pipelined-shuffle.compression.enabled: true
      

      But when I set taskmanager.numberOfTaskSlots to 4 or 6:

      taskmanager.numberOfTaskSlots: 6
      taskmanager.network.pipelined-shuffle.compression.enabled: true
      

      job failed:

      $ bin/flink run -m yarn-cluster -p 16 -yjm 1024m -ytm 12288m ~/flink-example-1.0-SNAPSHOT.jar
      SLF4J: Class path contains multiple SLF4J bindings.
      SLF4J: Found binding in [jar:file:/data/build/flink/flink-dist/target/flink-1.10-SNAPSHOT-bin/flink-1.10-SNAPSHOT/lib/slf4j-log4j12-1.7.15.jar!/org/slf4j/impl/StaticLoggerBinder.class]
      SLF4J: Found binding in [jar:file:/data/sa_cluster/cloudera/parcels/CDH-5.14.4-1.cdh5.14.4.p0.3/jars/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
      SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
      SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]
      2019-12-18 15:04:40,514 WARN  org.apache.flink.yarn.cli.FlinkYarnSessionCli                 - The configuration directory ('/data/build/flink/flink-dist/target/flink-1.10-SNAPSHOT-bin/flink-1.10-SNAPSHOT/conf') already contains a LOG4J config file.If you want to use logback, then please delete or rename the log configuration file.
      2019-12-18 15:04:40,514 WARN  org.apache.flink.yarn.cli.FlinkYarnSessionCli                 - The configuration directory ('/data/build/flink/flink-dist/target/flink-1.10-SNAPSHOT-bin/flink-1.10-SNAPSHOT/conf') already contains a LOG4J config file.If you want to use logback, then please delete or rename the log configuration file.
      2019-12-18 15:04:40,907 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - No path for the flink jar passed. Using the location of class org.apache.flink.yarn.YarnClusterDescriptor to locate the jar
      2019-12-18 15:04:41,084 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - Cluster specification: ClusterSpecification{masterMemoryMB=1024, taskManagerMemoryMB=12288, numberTaskManagers=1, slotsPerTaskManager=6}
      2019-12-18 15:04:42,344 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - Submitting application master application_1576573857638_0026
      2019-12-18 15:04:42,370 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Submitted application application_1576573857638_0026
      2019-12-18 15:04:42,371 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - Waiting for the cluster to be allocated
      2019-12-18 15:04:42,372 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - Deploying cluster, current state ACCEPTED
      2019-12-18 15:04:45,388 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - YARN application has been deployed successfully.
      2019-12-18 15:04:45,390 INFO  org.apache.flink.yarn.YarnClusterDescriptor                   - Found Web Interface debugboxcreate431x3.sa:36162 of application 'application_1576573857638_0026'.
      Job has been submitted with JobID 9140c70769f4271cc22ea8becaa26272
      
      ------------------------------------------------------------
       The program finished with the following exception:
      
      org.apache.flink.client.program.ProgramInvocationException: The main method caused an error: org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 9140c70769f4271cc22ea8becaa26272)
      	at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
      	at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
      	at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
      	at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
      	at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
      	at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
      	at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:422)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1924)
      	at org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
      	at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
      Caused by: java.util.concurrent.ExecutionException: org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 9140c70769f4271cc22ea8becaa26272)
      	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
      	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
      	at org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:83)
      	at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1628)
      	at cn.kbyte.StreamingJob.main(StreamingJob.java:247)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:321)
      	... 11 more
      Caused by: org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 9140c70769f4271cc22ea8becaa26272)
      	at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:112)
      	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
      	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
      	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
      	at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962)
      	at org.apache.flink.client.program.rest.RestClusterClient.lambda$pollResourceAsync$21(RestClusterClient.java:532)
      	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.complete(CompletableFuture.java:1962)
      	at org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$8(FutureUtils.java:291)
      	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.postFire(CompletableFuture.java:561)
      	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:929)
      	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
      	at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
      	at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:110)
      	... 19 more
      Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
      	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:110)
      	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:76)
      	at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:188)
      	at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:183)
      	at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:177)
      	at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:456)
      	at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:380)
      	at sun.reflect.GeneratedMethodAccessor24.invoke(Unknown Source)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:279)
      	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:194)
      	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
      	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
      	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
      	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
      	at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
      	at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
      	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
      	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
      	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
      	at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
      	at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
      	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
      	at akka.actor.ActorCell.invoke(ActorCell.scala:561)
      	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
      	at akka.dispatch.Mailbox.run(Mailbox.scala:225)
      	at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
      	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: org.apache.flink.runtime.io.compression.DataCorruptionException: Input is corrupted
      	at org.apache.flink.runtime.io.compression.Lz4BlockDecompressor.decompress(Lz4BlockDecompressor.java:80)
      	at org.apache.flink.runtime.io.network.buffer.BufferDecompressor.decompress(BufferDecompressor.java:97)
      	at org.apache.flink.runtime.io.network.buffer.BufferDecompressor.decompressToIntermediateBuffer(BufferDecompressor.java:60)
      	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.decompressBufferIfNeeded(SingleInputGate.java:581)
      	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.transformBuffer(SingleInputGate.java:543)
      	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.transformToBufferOrEvent(SingleInputGate.java:536)
      	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.getNextBufferOrEvent(SingleInputGate.java:492)
      	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.pollNext(SingleInputGate.java:474)
      	at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.pollNext(InputGateWithMetrics.java:75)
      	at org.apache.flink.streaming.runtime.io.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:125)
      	at org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:133)
      	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:311)
      	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:187)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:488)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:470)
      	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:702)
      	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:527)
      	at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.flink.runtime.shaded.net.jpountz.lz4.LZ4Exception: Malformed input at 1875
      	at org.apache.flink.runtime.shaded.net.jpountz.lz4.LZ4JavaUnsafeFastDecompressor.decompress(LZ4JavaUnsafeFastDecompressor.java:172)
      	at org.apache.flink.runtime.io.compression.Lz4BlockDecompressor.decompress(Lz4BlockDecompressor.java:65)
      	... 18 more
      

       

      Attachments

        Issue Links

          Activity

            People

              kevin.cyj Yingjie Cao
              fengjiajie Feng Jiajie
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 40m
                  40m