Skip to content
New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

[3.2][Spark] Execute MERGE using Dataframe API in Scala #3585

Merged
merged 1 commit into from
Aug 20, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,7 @@ class DeltaMergeBuilder private(
// Resolve UpCast expressions that `PreprocessTableMerge` may have introduced.
mergeIntoCommand = PostHocResolveUpCast(sparkSession).apply(mergeIntoCommand)
sparkSession.sessionState.analyzer.checkAnalysis(mergeIntoCommand)
mergeIntoCommand.asInstanceOf[MergeIntoCommand].run(sparkSession)
toDataset(sparkSession, mergeIntoCommand)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import java.util.Locale
import scala.language.implicitConversions

import com.databricks.spark.util.{Log4jUsageLogger, MetricDefinitions, UsageRecord}
import org.apache.spark.sql.delta.commands.MergeIntoCommand
import org.apache.spark.sql.delta.commands.merge.MergeStats
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
Expand Down Expand Up @@ -3327,6 +3328,25 @@ abstract class MergeIntoSuiteBase
"delta.dml.merge.findTouchedFiles",
"delta.dml.merge.writeInsertsOnlyWhenNoMatches",
"delta.dml.merge")

test("merge execution is recorded with QueryExecutionListener") {
withKeyValueData(
source = (0, 0) :: (1, 10) :: Nil,
target = (1, 1) :: (2, 2) :: Nil) { case (sourceName, targetName) =>
val plans = withLogicalPlansCaptured(spark, optimizedPlan = false) {
executeMerge(
tgt = s"$targetName t",
src = s"$sourceName s",
cond = "s.key = t.key",
update(set = "*"))
}
val mergeCommands = plans.collect {
case m: MergeIntoCommand => m
}
assert(mergeCommands.size === 1,
"Merge command wasn't properly recorded by QueryExecutionListener")
}
}
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,21 +26,15 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSparkSession

trait SchemaValidationSuiteBase extends QueryTest with SharedSparkSession with DeltaSQLCommandTest {

def checkMergeException(e: Exception, col: String): Unit = {
assert(e.isInstanceOf[MetadataChangedException])
assert(e.getMessage.contains(
"The metadata of the Delta table has been changed by a concurrent update"))
}
}

/**
* This Suite tests the behavior of Delta commands when a schema altering commit is run after the
* command completes analysis but before the command starts the transaction. We want to make sure
* That we do not corrupt tables.
*/
class SchemaValidationSuite extends SchemaValidationSuiteBase {
class SchemaValidationSuite
extends QueryTest
with SharedSparkSession
with DeltaSQLCommandTest {

class BlockingRule(
blockActionLatch: CountDownLatch,
Expand Down Expand Up @@ -331,7 +325,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {

/**
* Concurrently drop column in merge condition. Merge command detects the schema change while
* resolving the target and throws an AnalysisException
* resolving the target and throws a DeltaAnalysisException
*/
testConcurrentChange("merge - remove a column in merge condition concurrently")(
createTable = (spark: SparkSession, tblPath: String) => {
Expand All @@ -343,7 +337,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
actionToTest = (spark: SparkSession, tblPath: String) => {
val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath)
val sourceDf = spark.range(10).withColumn("col2", lit(2))
val e = intercept[Exception] {
val e = intercept[DeltaAnalysisException] {
deltaTable.as("t1")
.merge(sourceDf.as("t2"), "t1.id == t2.id")
.whenNotMatched()
Expand All @@ -352,14 +346,22 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
.updateAll()
.execute()
}
checkMergeException(e, "id")

checkErrorMatchPVals(
exception = e,
errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS",
parameters = Map(
"schemaDiff" -> ".*id.*",
"legacyFlagMessage" -> ""
)
)
},
concurrentChange = dropColFromSampleTable("id")
)

/**
* Concurrently drop column not in merge condition but in target. Merge command detects the schema
* change while resolving the target and throws an AnalysisException
* change while resolving the target and throws a DeltaAnalysisException
*/
testConcurrentChange("merge - remove a column not in merge condition concurrently")(
createTable = (spark: SparkSession, tblPath: String) => {
Expand All @@ -371,7 +373,7 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
actionToTest = (spark: SparkSession, tblPath: String) => {
val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath)
val sourceDf = spark.range(10).withColumn("col2", lit(2))
val e = intercept[Exception] {
val e = intercept[DeltaAnalysisException] {
deltaTable.as("t1")
.merge(sourceDf.as("t2"), "t1.id == t2.id")
.whenNotMatched()
Expand All @@ -380,7 +382,14 @@ class SchemaValidationSuite extends SchemaValidationSuiteBase {
.updateAll()
.execute()
}
checkMergeException(e, "col2")
checkErrorMatchPVals(
exception = e,
errorClass = "DELTA_SCHEMA_CHANGE_SINCE_ANALYSIS",
parameters = Map(
"schemaDiff" -> ".*col2.*",
"legacyFlagMessage" -> ""
)
)
},
concurrentChange = dropColFromSampleTable("col2")
)
Expand Down
Loading