Skip to content

Commit 9ca8e82

Browse files
authored
[Spark] Execute MERGE using Dataframe API in Scala (#3456)
## Description Due to Spark unfortunate behavior of resolving plan nodes it doesn't know, the `DeltaMergeInto` plan created when using the MERGE scala API needs to be manually resolved to ensure spark doesn't interfere with its analysis. This currently completely bypasses Spark's analysis as we then manually execute the MERGE command which has negatiev effects, e.g. the execution is not visible in QueryExecutionListener. This change addresses this issue, by executing the plan using the Dataframe API after it's manually resolved so that the command goes through the regular code path. Resolves #1521 ## How was this patch tested? Covered by existing tests.
1 parent eb719f8 commit 9ca8e82

File tree

3 files changed

+46
-17
lines changed

3 files changed

+46
-17
lines changed

spark/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -323,7 +323,7 @@ class DeltaMergeBuilder private(
323323
// Resolve UpCast expressions that `PreprocessTableMerge` may have introduced.
324324
mergeIntoCommand = PostHocResolveUpCast(sparkSession).apply(mergeIntoCommand)
325325
sparkSession.sessionState.analyzer.checkAnalysis(mergeIntoCommand)
326-
mergeIntoCommand.asInstanceOf[MergeIntoCommand].run(sparkSession)
326+
toDataset(sparkSession, mergeIntoCommand)
327327
}
328328
}
329329

spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import java.util.Locale
2323
import scala.language.implicitConversions
2424

2525
import com.databricks.spark.util.{Log4jUsageLogger, MetricDefinitions, UsageRecord}
26+
import org.apache.spark.sql.delta.commands.MergeIntoCommand
2627
import org.apache.spark.sql.delta.commands.merge.MergeStats
2728
import org.apache.spark.sql.delta.sources.DeltaSQLConf
2829
import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
@@ -3362,6 +3363,25 @@ abstract class MergeIntoSuiteBase
33623363
"delta.dml.merge.findTouchedFiles",
33633364
"delta.dml.merge.writeInsertsOnlyWhenNoMatches",
33643365
"delta.dml.merge")
3366+
3367+
test("merge execution is recorded with QueryExecutionListener") {
3368+
withKeyValueData(
3369+
source = (0, 0) :: (1, 10) :: Nil,
3370+
target = (1, 1) :: (2, 2) :: Nil) { case (sourceName, targetName) =>
3371+
val plans = withLogicalPlansCaptured(spark, optimizedPlan = false) {
3372+
executeMerge(
3373+
tgt = s"$targetName t",
3374+
src = s"$sourceName s",
3375+
cond = "s.key = t.key",
3376+
update(set = "*"))
3377+
}
3378+
val mergeCommands = plans.collect {
3379+
case m: MergeIntoCommand => m
3380+
}
3381+
assert(mergeCommands.size === 1,
3382+
"Merge command wasn't properly recorded by QueryExecutionListener")
3383+
}
3384+
}
33653385
}
33663386

33673387

spark/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala

Lines changed: 25 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -26,21 +26,15 @@ import org.apache.spark.sql.catalyst.rules.Rule
2626
import org.apache.spark.sql.functions._
2727
import org.apache.spark.sql.test.SharedSparkSession
2828

29-
trait SchemaValidationSuiteBase extends QueryTest with SharedSparkSession with DeltaSQLCommandTest {
30-
31-
def checkMergeException(e: Exception, col: String): Unit = {
32-
assert(e.isInstanceOf[MetadataChangedException])
33-
assert(e.getMessage.contains(
34-
"The metadata of the Delta table has been changed by a concurrent update"))
35-
}
36-
}
37-
3829
/**
3930
* This Suite tests the behavior of Delta commands when a schema altering commit is run after the
4031
* command completes analysis but before the command starts the transaction. We want to make sure
4132
* That we do not corrupt tables.
4233
*/
43-
class SchemaValidationSuite extends SchemaValidationSuiteBase {
34+
class SchemaValidationSuite
35+
extends QueryTest
36+
with SharedSparkSession
37+
with DeltaSQLCommandTest {
4438

4539
class BlockingRule(
4640
blockActionLatch: CountDownLatch,
@@ -331,7 +325,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
331325

332326
/**
333327
* Concurrently drop column in merge condition. Merge command detects the schema change while
334-
* resolving the target and throws an AnalysisException
328+
* resolving the target and throws a DeltaAnalysisException
335329
*/
336330
testConcurrentChange("merge - remove a column in merge condition concurrently")(
337331
createTable = (spark: SparkSession, tblPath: String) => {
@@ -343,7 +337,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
343337
actionToTest = (spark: SparkSession, tblPath: String) => {
344338
val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath)
345339
val sourceDf = spark.range(10).withColumn("col2", lit(2))
346-
val e = intercept[Exception] {
340+
val e = intercept[DeltaAnalysisException] {
347341
deltaTable.as("t1")
348342
.merge(sourceDf.as("t2"), "t1.id == t2.id")
349343
.whenNotMatched()
@@ -352,14 +346,22 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
352346
.updateAll()
353347
.execute()
354348
}
355-
checkMergeException(e, "id")
349+
350+
checkErrorMatchPVals(
351+
exception = e,
352+
errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS",
353+
parameters = Map(
354+
"schemaDiff" -> ".*id.*",
355+
"legacyFlagMessage" -> ""
356+
)
357+
)
356358
},
357359
concurrentChange = dropColFromSampleTable("id")
358360
)
359361

360362
/**
361363
* Concurrently drop column not in merge condition but in target. Merge command detects the schema
362-
* change while resolving the target and throws an AnalysisException
364+
* change while resolving the target and throws a DeltaAnalysisException
363365
*/
364366
testConcurrentChange("merge - remove a column not in merge condition concurrently")(
365367
createTable = (spark: SparkSession, tblPath: String) => {
@@ -371,7 +373,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
371373
actionToTest = (spark: SparkSession, tblPath: String) => {
372374
val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath)
373375
val sourceDf = spark.range(10).withColumn("col2", lit(2))
374-
val e = intercept[Exception] {
376+
val e = intercept[DeltaAnalysisException] {
375377
deltaTable.as("t1")
376378
.merge(sourceDf.as("t2"), "t1.id == t2.id")
377379
.whenNotMatched()
@@ -380,7 +382,14 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
380382
.updateAll()
381383
.execute()
382384
}
383-
checkMergeException(e, "col2")
385+
checkErrorMatchPVals(
386+
exception = e,
387+
errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS",
388+
parameters = Map(
389+
"schemaDiff" -> ".*col2.*",
390+
"legacyFlagMessage" -> ""
391+
)
392+
)
384393
},
385394
concurrentChange = dropColFromSampleTable("col2")
386395
)

0 commit comments

Comments
 (0)