Skip to content

Commit 7c352e9

Browse files
Adds tests for REPLACE WHERE SQL syntax
Spark 3.4 added RELACE WHERE SQL support for insert. This PR adds tests for the feature after upgrading to Spark 3.4. Closes #1737 GitOrigin-RevId: 8bf0e7423a6f0846d5f9ef4e637ee9ced9bef8d1
1 parent fe83c96 commit 7c352e9

File tree

1 file changed

+168
-0
lines changed

1 file changed

+168
-0
lines changed

core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala

Lines changed: 168 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3077,4 +3077,172 @@ class DeltaNameColumnMappingSuite extends DeltaSuite
30773077
insertedDF.filter(col("id") >= 6).union(otherDF))
30783078
}
30793079
}
3080+
3081+
test("replaceWhere SQL - partition column - dynamic filter") {
3082+
withTempDir { dir =>
3083+
// create partitioned table
3084+
spark.range(100).withColumn("part", 'id % 10)
3085+
.write
3086+
.format("delta")
3087+
.partitionBy("part")
3088+
.save(dir.toString)
3089+
3090+
// ans will be used to replace the entire contents of the table
3091+
val ans = spark.range(10)
3092+
.withColumn("part", lit(0))
3093+
3094+
ans.createOrReplaceTempView("replace")
3095+
sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE part >=0 SELECT * FROM replace")
3096+
checkAnswer(spark.read.format("delta").load(dir.toString), ans)
3097+
}
3098+
}
3099+
3100+
test("replaceWhere SQL - partition column - static filter") {
3101+
withTable("tbl") {
3102+
// create partitioned table
3103+
spark.range(100).withColumn("part", lit(0))
3104+
.write
3105+
.format("delta")
3106+
.partitionBy("part")
3107+
.saveAsTable("tbl")
3108+
3109+
val partEq1DF = spark.range(10, 20)
3110+
.withColumn("part", lit(1))
3111+
partEq1DF.write.format("delta").mode("append").saveAsTable("tbl")
3112+
3113+
3114+
val replacer = spark.range(10)
3115+
.withColumn("part", lit(0))
3116+
3117+
replacer.createOrReplaceTempView("replace")
3118+
sql(s"INSERT INTO tbl REPLACE WHERE part=0 SELECT * FROM replace")
3119+
checkAnswer(spark.read.format("delta").table("tbl"), replacer.union(partEq1DF))
3120+
}
3121+
}
3122+
3123+
test("replaceWhere SQL - data column - dynamic") {
3124+
withTable("tbl") {
3125+
// write table
3126+
spark.range(100).withColumn("col", lit(1))
3127+
.write
3128+
.format("delta")
3129+
.saveAsTable("tbl")
3130+
3131+
val colGt2DF = spark.range(100, 200)
3132+
.withColumn("col", lit(3))
3133+
3134+
colGt2DF.write
3135+
.format("delta")
3136+
.mode("append")
3137+
.saveAsTable("tbl")
3138+
3139+
val replacer = spark.range(10)
3140+
.withColumn("col", lit(1))
3141+
3142+
replacer.createOrReplaceTempView("replace")
3143+
sql(s"INSERT INTO tbl REPLACE WHERE col < 2 SELECT * FROM replace")
3144+
checkAnswer(
3145+
spark.read.format("delta").table("tbl"),
3146+
replacer.union(colGt2DF)
3147+
)
3148+
}
3149+
}
3150+
3151+
test("replaceWhere SQL - data column - static") {
3152+
withTempDir { dir =>
3153+
// write table
3154+
spark.range(100).withColumn("col", lit(2))
3155+
.write
3156+
.format("delta")
3157+
.save(dir.toString)
3158+
3159+
val colEq2DF = spark.range(100, 200)
3160+
.withColumn("col", lit(1))
3161+
3162+
colEq2DF.write
3163+
.format("delta")
3164+
.mode("append")
3165+
.save(dir.toString)
3166+
3167+
val replacer = spark.range(10)
3168+
.withColumn("col", lit(2))
3169+
3170+
replacer.createOrReplaceTempView("replace")
3171+
sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE col = 2 SELECT * FROM replace")
3172+
checkAnswer(
3173+
spark.read.format("delta").load(dir.toString),
3174+
replacer.union(colEq2DF)
3175+
)
3176+
}
3177+
}
3178+
3179+
test("replaceWhere SQL - multiple predicates - static") {
3180+
withTempDir { dir =>
3181+
// write table
3182+
spark.range(100).withColumn("col", lit(2))
3183+
.write
3184+
.format("delta")
3185+
.save(dir.toString)
3186+
3187+
spark.range(100, 200).withColumn("col", lit(5))
3188+
.write
3189+
.format("delta")
3190+
.mode("append")
3191+
.save(dir.toString)
3192+
3193+
val colEq2DF = spark.range(100, 200)
3194+
.withColumn("col", lit(1))
3195+
3196+
colEq2DF.write
3197+
.format("delta")
3198+
.mode("append")
3199+
.save(dir.toString)
3200+
3201+
val replacer = spark.range(10)
3202+
.withColumn("col", lit(2))
3203+
3204+
replacer.createOrReplaceTempView("replace")
3205+
sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE col = 2 OR col = 5 " +
3206+
s"SELECT * FROM replace")
3207+
checkAnswer(
3208+
spark.read.format("delta").load(dir.toString),
3209+
replacer.union(colEq2DF)
3210+
)
3211+
}
3212+
}
3213+
3214+
test("replaceWhere with less than predicate") {
3215+
withTempDir { dir =>
3216+
val insertedDF = spark.range(10).toDF()
3217+
3218+
insertedDF.write.format("delta").save(dir.toString)
3219+
3220+
val otherDF = spark.range(start = 0, end = 4).toDF()
3221+
otherDF.write.format("delta").mode("overwrite")
3222+
.option(DeltaOptions.REPLACE_WHERE_OPTION, "id < 6")
3223+
.save(dir.toString)
3224+
checkAnswer(spark.read.load(dir.toString),
3225+
insertedDF.filter(col("id") >= 6).union(otherDF))
3226+
}
3227+
}
3228+
3229+
test("replaceWhere SQL with less than predicate") {
3230+
withTempDir { dir =>
3231+
val insertedDF = spark.range(10).toDF()
3232+
3233+
insertedDF.write.format("delta").save(dir.toString)
3234+
3235+
val otherDF = spark.range(start = 0, end = 4).toDF()
3236+
otherDF.createOrReplaceTempView("replace")
3237+
3238+
sql(
3239+
s"""
3240+
|INSERT INTO delta.`${dir.getAbsolutePath}`
3241+
|REPLACE WHERE id < 6
3242+
|SELECT * FROM replace
3243+
|""".stripMargin)
3244+
checkAnswer(spark.read.load(dir.toString),
3245+
insertedDF.filter(col("id") >= 6).union(otherDF))
3246+
}
3247+
}
30803248
}

0 commit comments

Comments
 (0)