[
https://issues.apache.org/jira/browse/FLINK-23235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17377240#comment-17377240
]
Guowei Ma commented on FLINK-23235:
-----------------------------------
Investigated the reason for this problem: At present, an operator may still
complete the checkpoint after receiving "EndOfInput". Leading to GlobalCommit
will submit an "end of input".Specifically, when Checkpoint4 is successful, the
source node has sent the end-of-input message
!screenshot-1.png!
Because at present, it is not certain that an operator can complete a
checkpoint after receiving "EndOfInput". Therefore, in Streaming mode, I will
tell the test sink not to output the string "end of input" in endOfInput.
In the future, this will be restored when Final-CP becomes a certainty.
> SinkITCase.writerAndCommitterAndGlobalCommitterExecuteInStreamingMode fails
> on azure
> ------------------------------------------------------------------------------------
>
> Key: FLINK-23235
> URL: https://issues.apache.org/jira/browse/FLINK-23235
> Project: Flink
> Issue Type: Bug
> Components: API / DataStream
> Affects Versions: 1.13.1
> Reporter: Xintong Song
> Priority: Major
> Labels: test-stability
> Fix For: 1.13.2
>
> Attachments: screenshot-1.png
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=19867&view=logs&j=02c4e775-43bf-5625-d1cc-542b5209e072&t=e5961b24-88d9-5c77-efd3-955422674c25&l=9972
> {code}
> Jul 03 23:57:29 [ERROR] Tests run: 6, Failures: 1, Errors: 0, Skipped: 0,
> Time elapsed: 5.53 s <<< FAILURE! - in
> org.apache.flink.test.streaming.runtime.SinkITCase
> Jul 03 23:57:29 [ERROR]
> writerAndCommitterAndGlobalCommitterExecuteInStreamingMode(org.apache.flink.test.streaming.runtime.SinkITCase)
> Time elapsed: 0.68 s <<< FAILURE!
> Jul 03 23:57:29 java.lang.AssertionError:
> Jul 03 23:57:29
> Jul 03 23:57:29 Expected: iterable over ["(895,null,-9223372036854775808)",
> "(895,null,-9223372036854775808)", "(127,null,-9223372036854775808)",
> "(127,null,-9223372036854775808)", "(148,null,-9223372036854775808)",
> "(148,null,-9223372036854775808)", "(161,null,-9223372036854775808)",
> "(161,null,-9223372036854775808)", "(148,null,-9223372036854775808)",
> "(148,null,-9223372036854775808)", "(662,null,-9223372036854775808)",
> "(662,null,-9223372036854775808)", "(822,null,-9223372036854775808)",
> "(822,null,-9223372036854775808)", "(491,null,-9223372036854775808)",
> "(491,null,-9223372036854775808)", "(275,null,-9223372036854775808)",
> "(275,null,-9223372036854775808)", "(122,null,-9223372036854775808)",
> "(122,null,-9223372036854775808)", "(850,null,-9223372036854775808)",
> "(850,null,-9223372036854775808)", "(630,null,-9223372036854775808)",
> "(630,null,-9223372036854775808)", "(682,null,-9223372036854775808)",
> "(682,null,-9223372036854775808)", "(765,null,-9223372036854775808)",
> "(765,null,-9223372036854775808)", "(434,null,-9223372036854775808)",
> "(434,null,-9223372036854775808)", "(970,null,-9223372036854775808)",
> "(970,null,-9223372036854775808)", "(714,null,-9223372036854775808)",
> "(714,null,-9223372036854775808)", "(795,null,-9223372036854775808)",
> "(795,null,-9223372036854775808)", "(288,null,-9223372036854775808)",
> "(288,null,-9223372036854775808)", "(422,null,-9223372036854775808)",
> "(422,null,-9223372036854775808)"] in any order
> Jul 03 23:57:29 but: Not matched: "end of input"
> Jul 03 23:57:29 at
> org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20)
> Jul 03 23:57:29 at
> org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:8)
> Jul 03 23:57:29 at
> org.apache.flink.test.streaming.runtime.SinkITCase.writerAndCommitterAndGlobalCommitterExecuteInStreamingMode(SinkITCase.java:139)
> Jul 03 23:57:29 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
> Method)
> Jul 03 23:57:29 at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> Jul 03 23:57:29 at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> Jul 03 23:57:29 at java.lang.reflect.Method.invoke(Method.java:498)
> Jul 03 23:57:29 at
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
> Jul 03 23:57:29 at
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> Jul 03 23:57:29 at
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
> Jul 03 23:57:29 at
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> Jul 03 23:57:29 at
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> Jul 03 23:57:29 at
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
> Jul 03 23:57:29 at
> org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
> Jul 03 23:57:29 at
> org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
> Jul 03 23:57:29 at org.junit.rules.RunRules.evaluate(RunRules.java:20)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
> Jul 03 23:57:29 at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
> Jul 03 23:57:29 at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> Jul 03 23:57:29 at
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
> Jul 03 23:57:29 at
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
> Jul 03 23:57:29 at org.junit.rules.RunRules.evaluate(RunRules.java:20)
> Jul 03 23:57:29 at
> org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:365)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:273)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:238)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:159)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
> Jul 03 23:57:29 at
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)