You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Yingjie Cao (Jira)" <ji...@apache.org> on 2021/03/18 08:54:00 UTC

[jira] [Created] (FLINK-21857) StackOverflow for large parallelism jobs when processing EndOfChannelStateEvent

Yingjie Cao created FLINK-21857:
-----------------------------------

             Summary: StackOverflow for large parallelism jobs when processing EndOfChannelStateEvent
                 Key: FLINK-21857
                 URL: https://issues.apache.org/jira/browse/FLINK-21857
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Network
            Reporter: Yingjie Cao
             Fix For: 1.13.0


CheckpointedInputGate#handleEvent calls pollNext recursively when processing EndOfChannelStateEvent, for large parallelism job of large amount of input channels, the stack can become really deep thus causing StackOverflow. The following is the stack:
{code:java}
12:11
java.lang.StackOverflowError
  at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.waitAndGetNextData(SingleInputGate.java:650)
  at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.getNextBufferOrEvent(SingleInputGate.java:625)
  at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.pollNext(SingleInputGate.java:611)
  at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.pollNext(InputGateWithMetrics.java:109)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:149)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:158)
  at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.handleEvent(CheckpointedInputGate.java:202){code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)