Skip to content

Commit eee3467

Browse files
liwensunHyukjinKwon
authored andcommitted
[SPARK-27938][SQL] Remove feature flag LEGACY_PASS_PARTITION_BY_AS_OPTIONS
## What changes were proposed in this pull request? In PR #24365, we pass in the partitionBy columns as options in `DataFrameWriter`. To make this change less intrusive for a patch release, we added a feature flag `LEGACY_PASS_PARTITION_BY_AS_OPTIONS` with the default to be false. For 3.0, we should just do the correct behavior for DSV1, i.e., always passing partitionBy as options, and remove this legacy feature flag. ## How was this patch tested? Existing tests. Closes #24784 from liwensun/SPARK-27453-default. Authored-by: liwensun <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
1 parent 4d770db commit eee3467

File tree

3 files changed

+10
-30
lines changed

3 files changed

+10
-30
lines changed

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

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1711,15 +1711,6 @@ object SQLConf {
17111711
.booleanConf
17121712
.createWithDefault(false)
17131713

1714-
val LEGACY_PASS_PARTITION_BY_AS_OPTIONS =
1715-
buildConf("spark.sql.legacy.sources.write.passPartitionByAsOptions")
1716-
.internal()
1717-
.doc("Whether to pass the partitionBy columns as options in DataFrameWriter. " +
1718-
"Data source V1 now silently drops partitionBy columns for non-file-format sources; " +
1719-
"turning the flag on provides a way for these sources to see these partitionBy columns.")
1720-
.booleanConf
1721-
.createWithDefault(false)
1722-
17231714
val NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE =
17241715
buildConf("spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue")
17251716
.internal()

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

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -316,12 +316,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
316316
}
317317

318318
private def saveToV1Source(): Unit = {
319-
if (SparkSession.active.sessionState.conf.getConf(
320-
SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS)) {
321-
partitioningColumns.foreach { columns =>
322-
extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY ->
323-
DataSourceUtils.encodePartitioningColumns(columns))
324-
}
319+
partitioningColumns.foreach { columns =>
320+
extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY ->
321+
DataSourceUtils.encodePartitioningColumns(columns))
325322
}
326323

327324
// Code path for data source v1.

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

Lines changed: 7 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -225,21 +225,13 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be
225225
}
226226

227227
test("pass partitionBy as options") {
228-
Seq(true, false).foreach { flag =>
229-
withSQLConf(SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS.key -> s"$flag") {
230-
Seq(1).toDF.write
231-
.format("org.apache.spark.sql.test")
232-
.partitionBy("col1", "col2")
233-
.save()
234-
235-
if (flag) {
236-
val partColumns = LastOptions.parameters(DataSourceUtils.PARTITIONING_COLUMNS_KEY)
237-
assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2"))
238-
} else {
239-
assert(!LastOptions.parameters.contains(DataSourceUtils.PARTITIONING_COLUMNS_KEY))
240-
}
241-
}
242-
}
228+
Seq(1).toDF.write
229+
.format("org.apache.spark.sql.test")
230+
.partitionBy("col1", "col2")
231+
.save()
232+
233+
val partColumns = LastOptions.parameters(DataSourceUtils.PARTITIONING_COLUMNS_KEY)
234+
assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2"))
243235
}
244236

245237
test("save mode") {

0 commit comments

Comments
 (0)