[ 
https://issues.apache.org/jira/browse/FLINK-34416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17815738#comment-17815738
 ] 

Matthias Pohl commented on FLINK-34416:
---------------------------------------

We should investigate whether that could be fixed or whether it's a conceptual 
issue. We should add a proper explanation for why the tests are disabled for 
the AdaptiveScheduler in the code in the latter case.

> "Local recovery and sticky scheduling end-to-end test" still doesn't work 
> with AdaptiveScheduler
> ------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-34416
>                 URL: https://issues.apache.org/jira/browse/FLINK-34416
>             Project: Flink
>          Issue Type: Technical Debt
>          Components: Runtime / Coordination
>    Affects Versions: 1.19.0, 1.18.1, 1.20.0
>            Reporter: Matthias Pohl
>            Priority: Major
>              Labels: test-stability
>
> We tried to enable all {{AdaptiveScheduler}}-related tests in FLINK-34409 
> because it appeared that all Jira issues that were referenced are resolved. 
> That's not the case for the {{"Local recovery and sticky scheduling 
> end-to-end test"}} tests, though.
> With the {{AdaptiveScheduler}} being enabled, we run into issues where the 
> test runs forever due to a {{NullPointerException}} continuously triggering a 
> failure:
> {code}
> Feb 07 19:02:59 2024-02-07 19:02:21,706 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Flat Map -> 
> Sink: Unnamed (3/4) 
> (54075d3d22edb729e5f396726f777860_20ba6b65f97481d5570070de90e4e791_2_16292) 
> switched from INITIALIZING to FAILED on localhost:40893-09ff7>
> Feb 07 19:02:59 java.lang.NullPointerException: Expected to find info here.
> Feb 07 19:02:59         at 
> org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:76) 
> ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.tests.StickyAllocationAndLocalRecoveryTestJob$StateCreatingFlatMap.initializeState(StickyAllocationAndLocalRecoveryTestJob.java:340)
>  ~[?:?]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:187)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:169)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:134)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:285)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:106)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateAndGates(StreamTask.java:799)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$restoreInternal$3(StreamTask.java:753)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:753)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:712)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958)
>  ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927) 
> ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:751) 
> ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at 
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) 
> ~[flink-dist-1.20-SNAPSHOT.jar:1.20-SNAPSHOT]
> Feb 07 19:02:59         at java.lang.Thread.run(Thread.java:750) 
> ~[?:1.8.0_402]
> {code}
> This error is caused by a Precondition in 
> [StickyAllocationAndLocalRecoveryTestJob:340|https://github.com/apache/flink/blob/0f3470db83c1fddba9ac9a7299b1e61baab4ff12/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java#L340]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to