[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/14030 --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user lw-lin commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69856084 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala --- @@ -30,7 +32,42 @@ import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { override def addBatch(batchId: Long, data: DataFrame): Unit = { -data.as[T].foreachPartition { iter => +// TODO: Refine this method when SPARK-16264 is resolved; see comments below. --- End diff -- The logical plan of the created `IncrementalExecution` has to be `deserialized`[#L55](https://github.com/apache/spark/pull/14030/files#diff-98acda846a9dd63efc42e0957594e05dR55), so we should not re-use the `QueryExecution` passed in? --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user lw-lin commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69855749 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) + } + + ForeachSinkSuite.clear() + + // -- batch 1 --- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + expectedEventsForPartition0 = Seq( +ForeachSinkSuite.Open(partition = 0, version = 1), +ForeachSinkSuite.Process(value = 5), +ForeachSinkSuite.Process(value = 7), +ForeachSinkSuite.Close(None) + ) + expectedEventsForPartition1 = Seq( +ForeachSinkSuite.Open(partition = 1, version = 1), +ForeachSinkSuite.Process(value = 6), +ForeachSinkSuite.Process(value = 8), +ForeachSinkSuite.Close(None) + ) + + allEvents = ForeachSinkSuite.allEvents() assert(allEvents.size === 2) assert { allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) } + + query.stop() +} + } + + test("foreach() with `complete` output mode") { +withTempDir { checkpointDir => + val input = MemoryStream[Int] + + val query = input.toDS() +.groupBy().count().as[Long].map(_.toInt) +.writeStream +.option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("complete") --- End diff -- fixed; thanks! --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user lw-lin commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69855731 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) --- End diff -- fixed. @jaceklaskowski @zsxwing thanks! --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user lw-lin commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69855673 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -155,7 +155,7 @@ private[sql] object Dataset { class Dataset[T] private[sql]( @transient val sparkSession: SparkSession, @DeveloperApi @transient val queryExecution: QueryExecution, -encoder: Encoder[T]) +val encoder: Encoder[T]) --- End diff -- this change had been reverted; thanks! --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user lw-lin commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69855694 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala --- @@ -30,7 +32,42 @@ import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { override def addBatch(batchId: Long, data: DataFrame): Unit = { -data.as[T].foreachPartition { iter => +// TODO: Refine this method when SPARK-16264 is resolved; see comments below. + +// This logic should've been as simple as: +// ``` +// data.as[T].foreachPartition { iter => ... } +// ``` +// +// Unfortunately, doing that would just break the incremental planing. The reason is, +// `Dataset.foreachPartition()` would further call `Dataset.rdd()`, but `Dataset.rdd()` just +// does not support `IncrementalExecution`. +// +// So as a provisional fix, below we've made a special version of `Dataset` with its `rdd()` +// method supporting incremental planning. But in the long run, we should generally make newly +// created Datasets use `IncrementalExecution` where necessary (which is SPARK-16264 tries to +// resolve). + +val dataAsT = data.as[T] +val datasetWithIncrementalExecution = + new Dataset(data.sparkSession, dataAsT.logicalPlan, dataAsT.encoder) { --- End diff -- fixed; thanks! --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69824404 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) --- End diff -- yeah, or just `allEvents.toSet === Set(expectedEventsForPartition0, expectedEventsForPartition1)` --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69823441 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala --- @@ -30,7 +32,42 @@ import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { override def addBatch(batchId: Long, data: DataFrame): Unit = { -data.as[T].foreachPartition { iter => +// TODO: Refine this method when SPARK-16264 is resolved; see comments below. + +// This logic should've been as simple as: +// ``` +// data.as[T].foreachPartition { iter => ... } +// ``` +// +// Unfortunately, doing that would just break the incremental planing. The reason is, +// `Dataset.foreachPartition()` would further call `Dataset.rdd()`, but `Dataset.rdd()` just +// does not support `IncrementalExecution`. +// +// So as a provisional fix, below we've made a special version of `Dataset` with its `rdd()` +// method supporting incremental planning. But in the long run, we should generally make newly +// created Datasets use `IncrementalExecution` where necessary (which is SPARK-16264 tries to +// resolve). + +val dataAsT = data.as[T] +val datasetWithIncrementalExecution = + new Dataset(data.sparkSession, dataAsT.logicalPlan, dataAsT.encoder) { --- End diff -- `dataAsT` can be removed. You can use `implicitly[Encoder[T]]` to get the encoder. Please also revert the change to Dataset. --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user marmbrus commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69819131 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala --- @@ -30,7 +32,42 @@ import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { override def addBatch(batchId: Long, data: DataFrame): Unit = { -data.as[T].foreachPartition { iter => +// TODO: Refine this method when SPARK-16264 is resolved; see comments below. --- End diff -- Could we just use the QueryExecution of the dataframe that is passed in, instead of creating a new one? --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user marmbrus commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69819064 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -155,7 +155,7 @@ private[sql] object Dataset { class Dataset[T] private[sql]( @transient val sparkSession: SparkSession, @DeveloperApi @transient val queryExecution: QueryExecution, -encoder: Encoder[T]) +val encoder: Encoder[T]) --- End diff -- I'm not sure we want to make this publicly available. --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user jaceklaskowski commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69382676 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) + } + + ForeachSinkSuite.clear() + + // -- batch 1 --- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + expectedEventsForPartition0 = Seq( +ForeachSinkSuite.Open(partition = 0, version = 1), +ForeachSinkSuite.Process(value = 5), +ForeachSinkSuite.Process(value = 7), +ForeachSinkSuite.Close(None) + ) + expectedEventsForPartition1 = Seq( +ForeachSinkSuite.Open(partition = 1, version = 1), +ForeachSinkSuite.Process(value = 6), +ForeachSinkSuite.Process(value = 8), +ForeachSinkSuite.Close(None) + ) + + allEvents = ForeachSinkSuite.allEvents() assert(allEvents.size === 2) assert { allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) } + + query.stop() +} + } + + test("foreach() with `complete` output mode") { +withTempDir { checkpointDir => + val input = MemoryStream[Int] + + val query = input.toDS() +.groupBy().count().as[Long].map(_.toInt) +.writeStream +.option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("complete") --- End diff -- Are really output modes strings? No enums or similar more type-safe values? --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user jaceklaskowski commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69382669 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) + } + + ForeachSinkSuite.clear() + + // -- batch 1 --- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + expectedEventsForPartition0 = Seq( +ForeachSinkSuite.Open(partition = 0, version = 1), +ForeachSinkSuite.Process(value = 5), +ForeachSinkSuite.Process(value = 7), +ForeachSinkSuite.Close(None) + ) + expectedEventsForPartition1 = Seq( +ForeachSinkSuite.Open(partition = 1, version = 1), +ForeachSinkSuite.Process(value = 6), +ForeachSinkSuite.Process(value = 8), +ForeachSinkSuite.Close(None) + ) + + allEvents = ForeachSinkSuite.allEvents() assert(allEvents.size === 2) assert { allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) --- End diff -- Same as above --- 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
[GitHub] spark pull request #14030: [SPARK-16350][SQL] Fix support for incremental pl...
Github user jaceklaskowski commented on a diff in the pull request: https://github.com/apache/spark/pull/14030#discussion_r69382667 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala --- @@ -35,35 +35,109 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) +.outputMode("append") .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { +allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) --- End diff -- `should contain theSameElementsAs`? See http://www.scalatest.org/user_guide/using_matchers#workingWithAggregations --- 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