From ed40d14b1cba9c408636024dd4fe15dc9318346c Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 27 Jan 2022 17:53:37 +0900 Subject: [PATCH] [SPARK-35693][SS][TEST] More strict validation on plan check for stream-stream join unit test --- .../spark/sql/streaming/StreamingJoinSuite.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index a24e76f81b4aa..5ec47bb2aa527 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -28,6 +28,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreProviderId} @@ -583,9 +585,21 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { CheckAnswer(1.to(1000): _*), Execute { query => // Verify the query plan + def partitionExpressionsColumns(expressions: Seq[Expression]): Seq[String] = { + expressions.flatMap { + case ref: AttributeReference => Some(ref.name) + } + } + + val numPartitions = spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS) + assert(query.lastExecution.executedPlan.collect { case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _, - _: ShuffleExchangeExec, _: ShuffleExchangeExec) => j + ShuffleExchangeExec(opA: HashPartitioning, _, _), + ShuffleExchangeExec(opB: HashPartitioning, _, _)) + if partitionExpressionsColumns(opA.expressions) === Seq("a", "b") + && partitionExpressionsColumns(opB.expressions) === Seq("a", "b") + && opA.numPartitions == numPartitions && opB.numPartitions == numPartitions => j }.size == 1) }) }