Skip to content

Commit c8159c7

Browse files
brkyvzcloud-fan
authored andcommitted
[SPARK-29197][SQL] Remove saveModeForDSV2 from DataFrameWriter
### What changes were proposed in this pull request? It is very confusing that the default save mode is different between the internal implementation of a Data source. The reason that we had to have saveModeForDSV2 was that there was no easy way to check the existence of a Table in DataSource v2. Now, we have catalogs for that. Therefore we should be able to remove the different save modes. We also have a plan forward for `save`, where we can't really check the existence of a table, and therefore create one. That will come in a future PR. ### Why are the changes needed? Because it is confusing that the internal implementation of a data source (which is generally non-obvious to users) decides which default save mode is used within Spark. ### Does this PR introduce any user-facing change? It changes the default save mode for V2 Tables in the DataFrameWriter APIs ### How was this patch tested? Existing tests Closes #25876 from brkyvz/removeSM. Lead-authored-by: Burak Yavuz <brkyvz@gmail.com> Co-authored-by: Burak Yavuz <burak@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent b8b59d6 commit c8159c7

File tree

7 files changed

+51
-45
lines changed

7 files changed

+51
-45
lines changed

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -400,6 +400,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {
400400
.format("kafka")
401401
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
402402
.option("topic", topic)
403+
.mode("append")
403404
.save()
404405
checkAnswer(
405406
createKafkaReader(topic, includeHeaders = true).selectExpr(
@@ -423,6 +424,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {
423424
df.write
424425
.format("kafka")
425426
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
427+
.mode("append")
426428
.save()
427429
}
428430
TestUtils.assertExceptionMsg(ex, "null topic present in the data")
@@ -457,6 +459,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {
457459
.format("kafka")
458460
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
459461
.option("topic", topic)
462+
.mode("append")
460463
.save()
461464
}
462465
}

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1643,7 +1643,7 @@ object SQLConf {
16431643
"implementation class names for which Data Source V2 code path is disabled. These data " +
16441644
"sources will fallback to Data Source V1 code path.")
16451645
.stringConf
1646-
.createWithDefault("")
1646+
.createWithDefault("kafka")
16471647

16481648
val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers")
16491649
.doc("A comma-separated list of fully qualified data source register class names for which" +

sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala

Lines changed: 15 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
6767
* @since 1.4.0
6868
*/
6969
def mode(saveMode: SaveMode): DataFrameWriter[T] = {
70-
this.mode = Some(saveMode)
70+
this.mode = saveMode
7171
this
7272
}
7373

@@ -267,7 +267,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
267267
"if partition columns are specified.")
268268
}
269269
lazy val relation = DataSourceV2Relation.create(table, dsOptions)
270-
modeForDSV2 match {
270+
mode match {
271271
case SaveMode.Append =>
272272
runCommand(df.sparkSession, "save") {
273273
AppendData.byName(relation, df.logicalPlan, extraOptions.toMap)
@@ -308,7 +308,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
308308
sparkSession = df.sparkSession,
309309
className = source,
310310
partitionColumns = partitioningColumns.getOrElse(Nil),
311-
options = extraOptions.toMap).planForWriting(modeForDSV1, df.logicalPlan)
311+
options = extraOptions.toMap).planForWriting(mode, df.logicalPlan)
312312
}
313313
}
314314

@@ -319,6 +319,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
319319
* @note Unlike `saveAsTable`, `insertInto` ignores the column names and just uses position-based
320320
* resolution. For example:
321321
*
322+
* @note SaveMode.ErrorIfExists and SaveMode.Ignore behave as SaveMode.Append in `insertInto` as
323+
* `insertInto` is not a table creating operation.
324+
*
322325
* {{{
323326
* scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1")
324327
* scala> Seq((3, 4)).toDF("j", "i").write.insertInto("t1")
@@ -380,8 +383,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
380383
DataSourceV2Relation.create(t)
381384
}
382385

383-
val command = modeForDSV2 match {
384-
case SaveMode.Append =>
386+
val command = mode match {
387+
case SaveMode.Append | SaveMode.ErrorIfExists | SaveMode.Ignore =>
385388
AppendData.byPosition(table, df.logicalPlan, extraOptions.toMap)
386389

387390
case SaveMode.Overwrite =>
@@ -394,10 +397,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
394397
} else {
395398
OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap)
396399
}
397-
398-
case other =>
399-
throw new AnalysisException(s"insertInto does not support $other mode, " +
400-
s"please use Append or Overwrite mode instead.")
401400
}
402401

403402
runCommand(df.sparkSession, "insertInto") {
@@ -411,7 +410,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
411410
table = UnresolvedRelation(tableIdent),
412411
partitionSpec = Map.empty[String, Option[String]],
413412
query = df.logicalPlan,
414-
overwrite = modeForDSV1 == SaveMode.Overwrite,
413+
overwrite = mode == SaveMode.Overwrite,
415414
ifPartitionNotExists = false)
416415
}
417416
}
@@ -490,12 +489,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
490489

491490
session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match {
492491
case CatalogObjectIdentifier(Some(catalog), ident) =>
493-
saveAsTable(catalog.asTableCatalog, ident, modeForDSV2)
492+
saveAsTable(catalog.asTableCatalog, ident)
494493

495494
case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 =>
496-
// We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility
497-
// for now.
498-
saveAsTable(sessionCatalog.asTableCatalog, ident, modeForDSV1)
495+
saveAsTable(sessionCatalog.asTableCatalog, ident)
499496

500497
case AsTableIdentifier(tableIdentifier) =>
501498
saveAsTable(tableIdentifier)
@@ -507,7 +504,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
507504
}
508505

509506

510-
private def saveAsTable(catalog: TableCatalog, ident: Identifier, mode: SaveMode): Unit = {
507+
private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = {
511508
val partitioning = partitioningColumns.map { colNames =>
512509
colNames.map(name => IdentityTransform(FieldReference(name)))
513510
}.getOrElse(Seq.empty[Transform])
@@ -568,7 +565,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
568565
val tableIdentWithDB = tableIdent.copy(database = Some(db))
569566
val tableName = tableIdentWithDB.unquotedString
570567

571-
(tableExists, modeForDSV1) match {
568+
(tableExists, mode) match {
572569
case (true, SaveMode.Ignore) =>
573570
// Do nothing
574571

@@ -624,7 +621,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
624621
bucketSpec = getBucketSpec)
625622

626623
runCommand(df.sparkSession, "saveAsTable")(
627-
CreateTable(tableDesc, modeForDSV1, Some(df.logicalPlan)))
624+
CreateTable(tableDesc, mode, Some(df.logicalPlan)))
628625
}
629626

630627
/**
@@ -830,10 +827,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
830827
SQLExecution.withNewExecutionId(session, qe, Some(name))(qe.toRdd)
831828
}
832829

833-
private def modeForDSV1 = mode.getOrElse(SaveMode.ErrorIfExists)
834-
835-
private def modeForDSV2 = mode.getOrElse(SaveMode.Append)
836-
837830
private def lookupV2Provider(): Option[TableProvider] = {
838831
DataSource.lookupDataSourceV2(source, df.sparkSession.sessionState.conf) match {
839832
// TODO(SPARK-28396): File source v2 write path is currently broken.
@@ -848,7 +841,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
848841

849842
private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName
850843

851-
private var mode: Option[SaveMode] = None
844+
private var mode: SaveMode = SaveMode.ErrorIfExists
852845

853846
private val extraOptions = new scala.collection.mutable.HashMap[String, String]
854847

sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.spark.sql.connector
1919

2020
import org.apache.spark.sql.{DataFrame, Row, SaveMode}
21+
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
2122

2223
class DataSourceV2DataFrameSuite
2324
extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) {
@@ -75,13 +76,15 @@ class DataSourceV2DataFrameSuite
7576
withTable(t1) {
7677
sql(s"CREATE TABLE $t1 (id bigint, data string) USING foo")
7778
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
78-
// Default saveMode is append, therefore this doesn't throw a table already exists exception
79-
df.write.saveAsTable(t1)
79+
// Default saveMode is ErrorIfExists
80+
intercept[TableAlreadyExistsException] {
81+
df.write.saveAsTable(t1)
82+
}
83+
assert(spark.table(t1).count() === 0)
84+
85+
// appends are by name not by position
86+
df.select('data, 'id).write.mode("append").saveAsTable(t1)
8087
checkAnswer(spark.table(t1), df)
81-
82-
// also appends are by name not by position
83-
df.select('data, 'id).write.saveAsTable(t1)
84-
checkAnswer(spark.table(t1), df.union(df))
8588
}
8689
}
8790

sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -225,8 +225,12 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession {
225225
spark.read.format(cls.getName).option("path", path).load(),
226226
spark.range(10).select('id, -'id))
227227

228-
// default save mode is append
229-
spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName)
228+
// default save mode is ErrorIfExists
229+
intercept[AnalysisException] {
230+
spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName)
231+
.option("path", path).save()
232+
}
233+
spark.range(10).select('id as 'i, -'id as 'j).write.mode("append").format(cls.getName)
230234
.option("path", path).save()
231235
checkAnswer(
232236
spark.read.format(cls.getName).option("path", path).load(),
@@ -281,7 +285,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession {
281285

282286
val numPartition = 6
283287
spark.range(0, 10, 1, numPartition).select('id as 'i, -'id as 'j).write.format(cls.getName)
284-
.option("path", path).save()
288+
.mode("append").option("path", path).save()
285289
checkAnswer(
286290
spark.read.format(cls.getName).option("path", path).load(),
287291
spark.range(10).select('id, -'id))
@@ -368,7 +372,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession {
368372
val format = classOf[SimpleWritableDataSource].getName
369373

370374
val df = Seq((1L, 2L)).toDF("i", "j")
371-
df.write.format(format).option("path", optionPath).save()
375+
df.write.format(format).mode("append").option("path", optionPath).save()
372376
assert(!new File(sessionPath).exists)
373377
checkAnswer(spark.read.format(format).option("path", optionPath).load(), df)
374378
}

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ class NoopSuite extends SharedSparkSession {
3232
}
3333
.write
3434
.format("noop")
35+
.mode("append")
3536
.save()
3637
assert(accum.value == numElems)
3738
}
@@ -54,7 +55,7 @@ class NoopSuite extends SharedSparkSession {
5455
accum.add(1)
5556
x
5657
}
57-
.write.format("noop").save()
58+
.write.mode("append").format("noop").save()
5859
assert(accum.value == numElems)
5960
}
6061
}

sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala

Lines changed: 13 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -289,18 +289,20 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with
289289
assert(plan.isInstanceOf[OverwriteByExpression])
290290

291291
// By default the save mode is `ErrorIfExists` for data source v2.
292-
spark.range(10).write
293-
.format(classOf[NoopDataSource].getName)
294-
.save()
295-
sparkContext.listenerBus.waitUntilEmpty()
296-
assert(plan.isInstanceOf[AppendData])
292+
val e = intercept[AnalysisException] {
293+
spark.range(10).write
294+
.format(classOf[NoopDataSource].getName)
295+
.save()
296+
}
297+
assert(e.getMessage.contains("ErrorIfExists"))
297298

298-
spark.range(10).write
299-
.format(classOf[NoopDataSource].getName)
300-
.mode("default")
301-
.save()
302-
sparkContext.listenerBus.waitUntilEmpty()
303-
assert(plan.isInstanceOf[AppendData])
299+
val e2 = intercept[AnalysisException] {
300+
spark.range(10).write
301+
.format(classOf[NoopDataSource].getName)
302+
.mode("default")
303+
.save()
304+
}
305+
assert(e2.getMessage.contains("ErrorIfExists"))
304306
} finally {
305307
spark.listenerManager.unregister(listener)
306308
}

0 commit comments

Comments
 (0)