Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
6 changes: 6 additions & 0 deletions docs/sql-migration-guide-upgrade.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,12 @@ displayTitle: Spark SQL Upgrading Guide
* Table of contents
{:toc}

## Upgrading from Spark SQL 2.4 to 2.4.5

- Starting from 2.4.5, SQL configurations are effective also when a Dataset is converted to an RDD and its
plan is executed due to action on the derived RDD. The previous buggy behavior can be restored setting
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'd personally refrain from using the term buggy. Please explain what the previous behavior was.

`spark.sql.legacy.rdd.applyConf` to `false`.
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Hi, @hvanhovell , @cloud-fan and @gatorsmile . As @mgaido91 asked here, this PR will add this flag only at branch-2.4. In this case, is it okay this config will be added and deprecated at 2.4.5 and will be removed at 3.0.0?

For me, we don't need to add this configuration to master. Did I understand correctly?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I don't think we need to add the flag to master. We are allowed to break behavior there.


## Upgrading from Spark SQL 2.4 to 2.4.1

- The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1298,6 +1298,14 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val USE_CONF_ON_RDD_OPERATION =
buildConf("spark.sql.legacy.rdd.applyConf")
.internal()
.doc("When false, SQL configurations are disregarded when operations on a RDD derived from" +
" a dataframe are executed. This is the (buggy) behavior up to 2.4.3.")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

up to 2.4.4 ?

.booleanConf
.createWithDefault(true)

val REPLACE_EXCEPT_WITH_FILTER = buildConf("spark.sql.optimizer.replaceExceptWithFilter")
.internal()
.doc("When true, the apply function of the rule verifies whether the right node of the" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand}
import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _}
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -77,8 +78,13 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan)

/** Internal version of the RDD. Avoids copies and has no schema */
lazy val toRdd: RDD[InternalRow] = new SQLExecutionRDD(
executedPlan.execute(), sparkSession.sessionState.conf)
lazy val toRdd: RDD[InternalRow] = {
if (sparkSession.sessionState.conf.getConf(SQLConf.USE_CONF_ON_RDD_OPERATION)) {
new SQLExecutionRDD(executedPlan.execute(), sparkSession.sessionState.conf)
} else {
executedPlan.execute()
}
}

/**
* Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,6 @@
*/
package org.apache.spark.sql.execution

import java.util.Properties

import scala.collection.JavaConverters._

import org.apache.spark.{Partition, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
Expand All @@ -38,10 +34,8 @@ class SQLExecutionRDD(
var sqlRDD: RDD[InternalRow], conf: SQLConf) extends RDD[InternalRow](sqlRDD) {
private val sqlConfigs = conf.getAllConfs
private lazy val sqlConfExecutorSide = {
Comment thread
dongjoon-hyun marked this conversation as resolved.
val props = new Properties()
props.putAll(sqlConfigs.asJava)
val newConf = new SQLConf()
newConf.setConf(props)
sqlConfigs.foreach { case (k, v) => newConf.setConfString(k, v) }
newConf
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,17 +105,29 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils {
}

test("SPARK-28939: propagate SQLConf also in conversions to RDD") {
val confs = Seq("spark.sql.a" -> "x", "spark.sql.b" -> "y")
val physicalPlan = SQLConfAssertPlan(confs)
val dummyQueryExecution = FakeQueryExecution(spark, physicalPlan)
withSQLConf(confs: _*) {
// Force RDD evaluation to trigger asserts
dummyQueryExecution.toRdd.collect()
withSQLConf(SQLConf.USE_CONF_ON_RDD_OPERATION.key -> "true") {
val confs = Seq("spark.sql.a" -> "x", "spark.sql.b" -> "y")
val physicalPlan = SQLConfAssertPlan(confs)
val dummyQueryExecution = FakeQueryExecution(spark, physicalPlan)
withSQLConf(confs: _*) {
// Force RDD evaluation to trigger asserts
dummyQueryExecution.toRdd.collect()
}
val dummyQueryExecution1 = FakeQueryExecution(spark, physicalPlan)
// Without setting the configs assertions fail
val e = intercept[SparkException](dummyQueryExecution1.toRdd.collect())
assert(e.getCause.isInstanceOf[NoSuchElementException])
}
withSQLConf(SQLConf.USE_CONF_ON_RDD_OPERATION.key -> "false") {
val confs = Seq("spark.sql.a" -> "x", "spark.sql.b" -> "y")
val physicalPlan = SQLConfAssertPlan(confs)
val dummyQueryExecution = FakeQueryExecution(spark, physicalPlan)
withSQLConf(confs: _*) {
// Force RDD evaluation to trigger asserts
val e = intercept[SparkException](dummyQueryExecution.toRdd.collect())
assert(e.getCause.isInstanceOf[NoSuchElementException])
}
}
val dummyQueryExecution1 = FakeQueryExecution(spark, physicalPlan)
// Without setting the configs assertions fail
val e = intercept[SparkException](dummyQueryExecution1.toRdd.collect())
assert(e.getCause.isInstanceOf[NoSuchElementException])
}
}

Expand Down