diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b9e9e59bff54..583db58a67a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1711,15 +1711,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val LEGACY_PASS_PARTITION_BY_AS_OPTIONS = - buildConf("spark.sql.legacy.sources.write.passPartitionByAsOptions") - .internal() - .doc("Whether to pass the partitionBy columns as options in DataFrameWriter. " + - "Data source V1 now silently drops partitionBy columns for non-file-format sources; " + - "turning the flag on provides a way for these sources to see these partitionBy columns.") - .booleanConf - .createWithDefault(false) - val NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE = buildConf("spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 0c48ec9bb465..e900c185f824 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -316,12 +316,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveToV1Source(): Unit = { - if (SparkSession.active.sessionState.conf.getConf( - SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS)) { - partitioningColumns.foreach { columns => - extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY -> - DataSourceUtils.encodePartitioningColumns(columns)) - } + partitioningColumns.foreach { columns => + extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY -> + DataSourceUtils.encodePartitioningColumns(columns)) } // Code path for data source v1. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 5e6e3b4fc164..e9ab62800f84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -225,21 +225,13 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be } test("pass partitionBy as options") { - Seq(true, false).foreach { flag => - withSQLConf(SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS.key -> s"$flag") { - Seq(1).toDF.write - .format("org.apache.spark.sql.test") - .partitionBy("col1", "col2") - .save() - - if (flag) { - val partColumns = LastOptions.parameters(DataSourceUtils.PARTITIONING_COLUMNS_KEY) - assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2")) - } else { - assert(!LastOptions.parameters.contains(DataSourceUtils.PARTITIONING_COLUMNS_KEY)) - } - } - } + Seq(1).toDF.write + .format("org.apache.spark.sql.test") + .partitionBy("col1", "col2") + .save() + + val partColumns = LastOptions.parameters(DataSourceUtils.PARTITIONING_COLUMNS_KEY) + assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2")) } test("save mode") {