This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 643e8a9 [SPARK-38564][SS][TESTS] Wait all events to arrive in ReportSinkMetricsSuite 643e8a9 is described below commit 643e8a98bded006b743eb0388ca1414be005c439 Author: Hyukjin Kwon <gurwls...@apache.org> AuthorDate: Wed Mar 23 14:54:10 2022 +0900 [SPARK-38564][SS][TESTS] Wait all events to arrive in ReportSinkMetricsSuite ### What changes were proposed in this pull request? The test is flaky: ``` ReportSinkMetricsSuite: - test ReportSinkMetrics *** FAILED *** (244 milliseconds) Expected null, but got {"metrics-1"="value-1", "metrics-2"="value-2"} (ReportSinkMetricsSuite.scala:75) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472) at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471) at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563) at org.scalatest.Assertions.assertResult(Assertions.scala:867) at org.scalatest.Assertions.assertResult$(Assertions.scala:863) at org.scalatest.funsuite.AnyFunSuite.assertResult(AnyFunSuite.scala:1563) at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$2(ReportSinkMetricsSuite.scala:75) at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$2$adapted(ReportSinkMetricsSuite.scala:60) at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1(SQLTestUtils.scala:79) at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1$adapted(SQLTestUtils.scala:78) at org.apache.spark.SparkFunSuite.withTempDir(SparkFunSuite.scala:221) at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.org$apache$spark$sql$test$SQLTestUtils$$super$withTempDir(ReportSinkMetricsSuite.scala:35) at org.apache.spark.sql.test.SQLTestUtils.withTempDir(SQLTestUtils.scala:78) at org.apache.spark.sql.test.SQLTestUtils.withTempDir$(SQLTestUtils.scala:77) at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.withTempDir(ReportSinkMetricsSuite.scala:35) at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$1(ReportSinkMetricsSuite.scala:60) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) ``` We should wait all events to be processed. See https://github.com/apache/spark/pull/35872#discussion_r832807001. ### Why are the changes needed? To make the test not flaky. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Existing tests. CI in this PR should test it out. Closes #35945 from HyukjinKwon/SPARK-38564. Authored-by: Hyukjin Kwon <gurwls...@apache.org> Signed-off-by: Hyukjin Kwon <gurwls...@apache.org> --- .../scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala index 17aef18..c5bd389 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala @@ -72,6 +72,8 @@ class ReportSinkMetricsSuite extends StreamTest { query.processAllAvailable() } + spark.sparkContext.listenerBus.waitUntilEmpty() + assertResult(metricsMap) { Map("metrics-1" -> "value-1", "metrics-2" -> "value-2").asJava } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org