Sopan Phaltankar created FLINK-29789:
----------------------------------------

             Summary: Fix flaky tests in CheckpointCoordinatorTest
                 Key: FLINK-29789
                 URL: https://issues.apache.org/jira/browse/FLINK-29789
             Project: Flink
          Issue Type: Bug
            Reporter: Sopan Phaltankar


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)

Reply via email to