GitHub user tdas opened a pull request: https://github.com/apache/spark/pull/15519
[SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite This work has largely been done by @lw-lin in his PR #15497. This is a slightly refactoring of it. ## What changes were proposed in this pull request? There were two sources of flakiness in StreamingQueryListener test. - When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock. ``` +-----------------------------------+--------------------------------+ | StreamExecution thread | testing thread | +-----------------------------------+--------------------------------+ | ManualClock.waitTillTime(100) { | | | _isWaiting = true | | | wait(10) | | | still in wait(10) | if (_isWaiting) advance(100) | | still in wait(10) | if (_isWaiting) advance(200) | <- this should be disallowed ! | still in wait(10) | if (_isWaiting) advance(300) | <- this should be disallowed ! | wake up from wait(10) | | | current time is 600 | | | _isWaiting = false | | | } | | +-----------------------------------+--------------------------------+ ``` - Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger. My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(300)` (see above) will wait for stream execution thread to complete the wait that started at time 100, and start a new wait at time 300 (i.e. time stamp after the previous `advance(200)`). In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest. ## How was this patch tested? Ran existing unit test MANY TIME in Jenkins You can merge this pull request into a Git repository by running: $ git pull https://github.com/tdas/spark metrics-flaky-test-fix Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/15519.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #15519 ---- commit 5bc47b639ede049f44ad4f47a88d26219fea6193 Author: Liwei Lin <lwl...@gmail.com> Date: 2016-10-15T02:21:58Z Fix flaky test commit eb59a98146f30163675cec3b52f69fedd7a234fc Author: Liwei Lin <lwl...@gmail.com> Date: 2016-10-17T13:15:40Z Revert "Fix flaky test" This reverts commit 5bc47b639ede049f44ad4f47a88d26219fea6193. commit 7ae7782cdede0c3f2a3db0a09401cf0d682a264f Author: Liwei Lin <lwl...@gmail.com> Date: 2016-10-17T11:53:46Z Fix flaky test again commit 6fdbae34a6e806ad0ca8bb6cfd6ff630e0b84143 Author: Tathagata Das <tathagata.das1...@gmail.com> Date: 2016-10-17T20:46:48Z Refactored Manual clock ---- --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org