Skip to content
Closed
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 @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

decodePartitioningColumns is under execution package that's not supposed to be exposed so users shouldn't use this util directly.

Did we document this option to any public datasource v1 API? We should also say this is a JSON string.

}

test("save mode") {
Expand Down