Details
-
Bug
-
Status: Closed
-
Blocker
-
Resolution: Fixed
-
1.19.0
Description
This test failed in a master nightly workflow run in GitHub Actions (FLIP-396) which is based on master@fd673a2f4
https://github.com/XComp/flink/actions/runs/7632434859/job/20793612930#step:10:8625
Error: 01:07:53 01:07:53.367 [ERROR] Tests run: 12, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 16.85 s <<< FAILURE! -- in org.apache.flink.test.runtime.HybridShuffleITCase Error: 01:07:53 01:07:53.367 [ERROR] org.apache.flink.test.runtime.HybridShuffleITCase.testHybridSelectiveExchangesRestart -- Time elapsed: 1.164 s <<< FAILURE! Jan 24 01:07:53 java.lang.AssertionError: org.apache.flink.runtime.JobException: org.apache.flink.runtime.JobException: Recovery is suppressed by FixedDelayRestartBackoffTimeStrategy(maxNumberRestartAttempts=2, backoffTimeMS=0) Jan 24 01:07:53 at org.apache.flink.test.runtime.JobGraphRunningUtil.execute(JobGraphRunningUtil.java:59) Jan 24 01:07:53 at org.apache.flink.test.runtime.BatchShuffleITCaseBase.executeJob(BatchShuffleITCaseBase.java:137) Jan 24 01:07:53 at org.apache.flink.test.runtime.HybridShuffleITCase.testHybridSelectiveExchangesRestart(HybridShuffleITCase.java:91) Jan 24 01:07:53 at java.base/java.lang.reflect.Method.invoke(Method.java:580) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:184) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:179) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:184) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197) Jan 24 01:07:53 at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:1024) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:151) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:174) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276) Jan 24 01:07:53 at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1708) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:151) Jan 24 01:07:53 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:174) Jan 24 01:07:53 at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) Jan 24 01:07:53 at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596) Jan 24 01:07:53 at java.base/java.util.concurrent.RecursiveAction.exec(RecursiveAction.java:194) Jan 24 01:07:53 at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:387) Jan 24 01:07:53 at java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1312) Jan 24 01:07:53 at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1843) Jan 24 01:07:53 at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1808) Jan 24 01:07:53 at java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:188) Jan 24 01:07:53 Caused by: org.apache.flink.runtime.JobException: org.apache.flink.runtime.JobException: Recovery is suppressed by FixedDelayRestartBackoffTimeStrategy(maxNumberRestartAttempts=2, backoffTimeMS=0) Jan 24 01:07:53 at org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:176) Jan 24 01:07:53 at org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:107) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:277) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:268) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:261) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:770) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:747) Jan 24 01:07:53 at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:83) Jan 24 01:07:53 at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:488) Jan 24 01:07:53 at java.base/java.lang.reflect.Method.invoke(Method.java:580) Jan 24 01:07:53 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.lambda$handleRpcInvocation$1(PekkoRpcActor.java:309) Jan 24 01:07:53 at org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) Jan 24 01:07:53 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcInvocation(PekkoRpcActor.java:307) Jan 24 01:07:53 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcMessage(PekkoRpcActor.java:222) Jan 24 01:07:53 at org.apache.flink.runtime.rpc.pekko.FencedPekkoRpcActor.handleRpcMessage(FencedPekkoRpcActor.java:85) Jan 24 01:07:53 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleMessage(PekkoRpcActor.java:168) Jan 24 01:07:53 at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:33) Jan 24 01:07:53 at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:29) Jan 24 01:07:53 at scala.PartialFunction.applyOrElse(PartialFunction.scala:127) Jan 24 01:07:53 at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126) Jan 24 01:07:53 at org.apache.pekko.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:29) Jan 24 01:07:53 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175) Jan 24 01:07:53 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) Jan 24 01:07:53 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) Jan 24 01:07:53 at org.apache.pekko.actor.Actor.aroundReceive(Actor.scala:547) Jan 24 01:07:53 at org.apache.pekko.actor.Actor.aroundReceive$(Actor.scala:545) Jan 24 01:07:53 at org.apache.pekko.actor.AbstractActor.aroundReceive(AbstractActor.scala:229) Jan 24 01:07:53 at org.apache.pekko.actor.ActorCell.receiveMessage(ActorCell.scala:590) Jan 24 01:07:53 at org.apache.pekko.actor.ActorCell.invoke(ActorCell.scala:557) Jan 24 01:07:53 at org.apache.pekko.dispatch.Mailbox.processMailbox(Mailbox.scala:280) Jan 24 01:07:53 at org.apache.pekko.dispatch.Mailbox.run(Mailbox.scala:241) Jan 24 01:07:53 at org.apache.pekko.dispatch.Mailbox.exec(Mailbox.scala:253) Jan 24 01:07:53 ... 5 more Jan 24 01:07:53 Caused by: org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: Fatal error at remote task manager 'localhost/127.0.0.1:42319 [ e9ae6545-d016-40a8-934c-3e46e5605fec ] '. Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.CreditBasedPartitionRequestClientHandler.decodeMsg(CreditBasedPartitionRequestClientHandler.java:308) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.CreditBasedPartitionRequestClientHandler.channelRead(CreditBasedPartitionRequestClientHandler.java:191) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.NettyMessageClientDecoderDelegate.channelRead(NettyMessageClientDecoderDelegate.java:112) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:800) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:509) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) Jan 24 01:07:53 at java.base/java.lang.Thread.run(Thread.java:1583) Jan 24 01:07:53 Caused by: java.io.IOException: java.io.IOException: Cannot invoke "java.lang.Integer.intValue()" because "java.util.Queue.peek().f1" is null Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.PartitionRequestQueue.writeAndFlushNextMessageIfPossible(PartitionRequestQueue.java:357) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.PartitionRequestQueue.enqueueAvailableReader(PartitionRequestQueue.java:125) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.PartitionRequestQueue.userEventTriggered(PartitionRequestQueue.java:253) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:400) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:376) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:368) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:117) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:387) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:400) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:376) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:368) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.userEventTriggered(DefaultChannelPipeline.java:1428) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:396) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:376) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireUserEventTriggered(DefaultChannelPipeline.java:913) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.PartitionRequestQueue.lambda$notifyReaderNonEmpty$0(PartitionRequestQueue.java:93) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470) Jan 24 01:07:53 at org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:413) Jan 24 01:07:53 ... 3 more Jan 24 01:07:53 Caused by: java.lang.NullPointerException: java.lang.NullPointerException Jan 24 01:07:53 at org.apache.flink.runtime.io.network.partition.UnionResultSubpartitionView.peekNextBufferSubpartitionId(UnionResultSubpartitionView.java:92) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.CreditBasedSequenceNumberingViewReader.peekNextBufferSubpartitionId(CreditBasedSequenceNumberingViewReader.java:250) Jan 24 01:07:53 at org.apache.flink.runtime.io.network.netty.PartitionRequestQueue.writeAndFlushNextMessageIfPossible(PartitionRequestQueue.java:317) Jan 24 01:07:53 ... 22 more
Attachments
Attachments
Issue Links
- is caused by
-
FLINK-33743 Support consuming multiple subpartitions on a single channel
- Closed
- relates to
-
FLINK-34233 HybridShuffleITCase.testHybridSelectiveExchangesRestart failed due to a IllegalStateException
- Closed
- links to
- mentioned in
-
Page Loading...