You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Sopan Phaltankar (Jira)" <ji...@apache.org> on 2022/10/28 04:35:00 UTC

[jira] [Comment Edited] (FLINK-29611) Fix flaky tests in CoBroadcastWithNonKeyedOperatorTest

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

Sopan Phaltankar edited comment on FLINK-29611 at 10/28/22 4:34 AM:
--------------------------------------------------------------------

[~martijnvisser] 
This test depends on the order of iteration in `HashMap.entrySet()` and can fail for some orders. I am running this on my local machine. I have used a maven plugin NonDex, which can be used to identify such tests. One can reproduce it with the command `mvn edu.illinois:nondex-maven-plugun:1.1.2:nondex -Dtest=org.apache.flink.streaming.api.operators.co.CoBroadcastWithNonKeyedOperatorTest#testMultiStateSupport`. Even if the test was not failing during the daily jobs, it'd be good to not depend on the `HashMap.entrySet()` that gives back results in an undefined manner [JavaDoc|https://docs.oracle.com/javase/8/docs/api/java/util/HashMap.html#entrySet--]for this. Therefore, to remove non-determinism completely, we can do this change.


was (Author: JIRAUSER296865):
[~martijnvisser] 
This test depends on the order of iteration in `HashMap.entrySet()` and can fail for some orders. I am running this on my local machine. I have used a maven plugin NonDex, which can be used to identify such tests. One can reproduce it with the command `mvn edu.illinois:nondex-maven-plugun:1.1.2:nondex -Dtest=org.apache.flink.streaming.api.operators.co.CoBroadcastWithNonKeyedOperatorTest#testMultiStateSupport`. Even if the test was not failing during the daily jobs, it'd be good to not depend on the `HashMap.entrySet()` that gives back results in an undefined manner [JavaDoc|[https://docs.oracle.com/javase/8/docs/api/java/util/HashMap.html#entrySet--]] for this. Therefore, to remove non-determinism completely, we can do this change.

> Fix flaky tests in CoBroadcastWithNonKeyedOperatorTest
> ------------------------------------------------------
>
>                 Key: FLINK-29611
>                 URL: https://issues.apache.org/jira/browse/FLINK-29611
>             Project: Flink
>          Issue Type: Bug
>            Reporter: Sopan Phaltankar
>            Priority: Minor
>              Labels: pull-request-available
>
> The test _org.apache.flink.streaming.api.operators.co.CoBroadcastWithNonKeyedOperatorTest.testMultiStateSupport_ has the following failure:
> Failures:
> [ERROR]   CoBroadcastWithNonKeyedOperatorTest.testMultiStateSupport:74 
> Wrong Side Output: arrays first differed at element [0]; expected:<Record @ 15 : 9:key.6->6> but was:<Record @ 15 : 9:key.5->5>
> I used the tool [NonDex|https://github.com/TestingResearchIllinois/NonDex] to find this flaky test. 
> Command: mvn edu.illinois:nondex-maven-plugun:1.1.2:nondex -Dtest='Fully Qualified Test Name'
> I analyzed the assertion failure and found that the root cause is because the test method calls ctx.getBroadcastState(STATE_DESCRIPTOR).immutableEntries() which calls the entrySet() method of the underlying HashMap. entrySet() returns the entries in a non-deterministic way, causing the test to be flaky. 
> The fix would be to change _HashMap_ to _LinkedHashMap_ where the Map is getting initialized.
> On further analysis, it was found that the Map is getting initialized on line 53 of org.apache.flink.runtime.state.HeapBroadcastState class.
> After changing from HashMap to LinkedHashMap, the above test is passing.
> Edit: Upon making this change and running the CI, it was found that the tests org.apache.flink.api.datastream.DataStreamBatchExecutionITCase.batchKeyedBroadcastExecution and org.apache.flink.api.datastream.DataStreamBatchExecutionITCase.batchBroadcastExecution were failing. Upon further investigation, I found that these tests were also flaky and depended on the earlier made change.



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