@@ -30,7 +30,7 @@ import org.scalatest.{BeforeAndAfter, Tag}
3030
3131import org .apache .spark .SparkUnsupportedOperationException
3232import org .apache .spark .scheduler .ExecutorCacheTaskLocation
33- import org .apache .spark .sql .{DataFrame , Row , SparkSession }
33+ import org .apache .spark .sql .{AnalysisException , DataFrame , Row , SparkSession }
3434import org .apache .spark .sql .catalyst .expressions .{AttributeReference , Expression }
3535import org .apache .spark .sql .catalyst .plans .physical .HashPartitioning
3636import org .apache .spark .sql .execution .datasources .v2 .state .StateSourceOptions
@@ -352,6 +352,28 @@ abstract class StreamingInnerJoinBase extends StreamingJoinSuite {
352352 )
353353 }
354354
355+ // Stream-stream non-outer join produces the same behavior between Append mode and Update mode.
356+ // We only run a sanity test here rather than replicating the full Append mode test suite.
357+ test(" stream stream inner join with Update mode on non-time column" ) {
358+ val input1 = MemoryStream [Int ]
359+ val input2 = MemoryStream [Int ]
360+
361+ val df1 = input1.toDF().select($" value" as " key" , ($" value" * 2 ) as " leftValue" )
362+ val df2 = input2.toDF().select($" value" as " key" , ($" value" * 3 ) as " rightValue" )
363+ val joined = df1.join(df2, " key" )
364+
365+ testStream(joined, OutputMode .Update ())(
366+ AddData (input1, 1 ),
367+ CheckAnswer (),
368+ AddData (input2, 1 , 10 ),
369+ CheckNewAnswer ((1 , 2 , 3 )),
370+ AddData (input1, 10 ),
371+ CheckNewAnswer ((10 , 20 , 30 )),
372+ AddData (input2, 1 ),
373+ CheckNewAnswer ((1 , 2 , 3 ))
374+ )
375+ }
376+
355377 test(" stream stream inner join on windows - without watermark" ) {
356378 val input1 = MemoryStream [Int ]
357379 val input2 = MemoryStream [Int ]
@@ -669,7 +691,7 @@ abstract class StreamingInnerJoinBase extends StreamingJoinSuite {
669691 assert(query.lastExecution.executedPlan.collect {
670692 case j @ StreamingSymmetricHashJoinExec (_, _, _, _, _, _, _, _, _,
671693 ShuffleExchangeExec (opA : HashPartitioning , _, _, _),
672- ShuffleExchangeExec (opB : HashPartitioning , _, _, _))
694+ ShuffleExchangeExec (opB : HashPartitioning , _, _, _), _ )
673695 if partitionExpressionsColumns(opA.expressions) === Seq (" a" , " b" )
674696 && partitionExpressionsColumns(opB.expressions) === Seq (" a" , " b" )
675697 && opA.numPartitions == numPartitions && opB.numPartitions == numPartitions => j
@@ -1242,6 +1264,25 @@ abstract class StreamingOuterJoinBase extends StreamingJoinSuite {
12421264 import testImplicits ._
12431265 import org .apache .spark .sql .functions ._
12441266
1267+ Seq (" left_outer" , " right_outer" ).foreach { joinType =>
1268+ test(s " stream-stream $joinType join does not support Update mode " ) {
1269+ val input1 = MemoryStream [Int ]
1270+ val input2 = MemoryStream [Int ]
1271+
1272+ val df1 = input1.toDF().select($" value" as " key" , ($" value" * 2 ) as " leftValue" )
1273+ val df2 = input2.toDF().select($" value" as " key" , ($" value" * 3 ) as " rightValue" )
1274+ val joined = df1.join(df2, Seq (" key" ), joinType)
1275+
1276+ val e = intercept[AnalysisException ] {
1277+ testStream(joined, OutputMode .Update ())(
1278+ AddData (input1, 1 ),
1279+ CheckAnswer ()
1280+ )
1281+ }
1282+ assert(e.getMessage.contains(" is not supported in Update output mode" ))
1283+ }
1284+ }
1285+
12451286 test(" left outer early state exclusion on left" ) {
12461287 withTempDir { checkpointDir =>
12471288 val (leftInput, rightInput, joined) = setupWindowedJoinWithLeftCondition(" left_outer" )
@@ -1954,6 +1995,25 @@ abstract class StreamingOuterJoinSuite extends StreamingOuterJoinBase {
19541995@ SlowSQLTest
19551996abstract class StreamingFullOuterJoinBase extends StreamingJoinSuite {
19561997
1998+ import testImplicits ._
1999+
2000+ test(" stream-stream full outer join does not support Update mode" ) {
2001+ val input1 = MemoryStream [Int ]
2002+ val input2 = MemoryStream [Int ]
2003+
2004+ val df1 = input1.toDF().select($" value" as " key" , ($" value" * 2 ) as " leftValue" )
2005+ val df2 = input2.toDF().select($" value" as " key" , ($" value" * 3 ) as " rightValue" )
2006+ val joined = df1.join(df2, Seq (" key" ), " full_outer" )
2007+
2008+ val e = intercept[AnalysisException ] {
2009+ testStream(joined, OutputMode .Update ())(
2010+ AddData (input1, 1 ),
2011+ CheckAnswer ()
2012+ )
2013+ }
2014+ assert(e.getMessage.contains(" is not supported in Update output mode" ))
2015+ }
2016+
19572017 test(" windowed full outer join" ) {
19582018 withTempDir { checkpointDir =>
19592019 val (leftInput, rightInput, joined) = setupWindowedJoin(" full_outer" )
@@ -2176,6 +2236,22 @@ abstract class StreamingLeftSemiJoinBase extends StreamingJoinSuite {
21762236
21772237 import testImplicits ._
21782238
2239+ // Stream-stream non-outer join produces the same behavior between Append mode and Update mode.
2240+ // We only run a sanity test here rather than replicating the full Append mode test suite.
2241+ test(" windowed left semi join with Update mode" ) {
2242+ withTempDir { checkpointDir =>
2243+ val (leftInput, rightInput, joined) = setupWindowedJoin(" left_semi" )
2244+
2245+ testStream(joined, OutputMode .Update ())(
2246+ StartStream (checkpointLocation = checkpointDir.getCanonicalPath),
2247+ MultiAddData (leftInput, 1 , 2 , 3 , 4 , 5 )(rightInput, 3 , 4 , 5 , 6 , 7 ),
2248+ CheckNewAnswer (Row (3 , 10 , 6 ), Row (4 , 10 , 8 ), Row (5 , 10 , 10 )),
2249+ MultiAddData (leftInput, 21 )(rightInput, 22 ),
2250+ CheckNewAnswer ()
2251+ )
2252+ }
2253+ }
2254+
21792255 test(" windowed left semi join" ) {
21802256 withTempDir { checkpointDir =>
21812257 val (leftInput, rightInput, joined) = setupWindowedJoin(" left_semi" )
0 commit comments