Skip to content

[SPARK-28939][SQL][2.4] Propagate SQLConf for plans executed by toRdd #25734

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 5 commits into from
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
7 changes: 7 additions & 0 deletions docs/sql-migration-guide-upgrade.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,13 @@ 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 behavior can be restored setting
`spark.sql.legacy.rdd.applyConf` to `false`: in this case, SQL configurations are ignored for operations
performed on a RDD derived from a Dataset.

## 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 @@ -113,7 +113,9 @@ object SQLConf {
* Returns the active config object within the current scope. If there is an active SparkSession,
* the proper SQLConf associated with the thread's active session is used. If it's called from
* tasks in the executor side, a SQLConf will be created from job local properties, which are set
* and propagated from the driver side.
* and propagated from the driver side, unless a `SQLConf` has been set in the scope by
* `withExistingConf` as done for propagating SQLConf for operations performed on RDDs created
* from DataFrames.
*
* The way this works is a little bit convoluted, due to the fact that config was added initially
* only for physical plans (and as a result not in sql/catalyst module).
Expand All @@ -127,7 +129,12 @@ object SQLConf {
*/
def get: SQLConf = {
if (TaskContext.get != null) {
new ReadOnlySQLConf(TaskContext.get())
val conf = existingConf.get()
if (conf != null) {
conf
} else {
new ReadOnlySQLConf(TaskContext.get())
}
} else {
val isSchedulerEventLoopThread = SparkContext.getActive
.map(_.dagScheduler.eventProcessLoop.eventThread)
Expand Down Expand Up @@ -1291,6 +1298,15 @@ 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.4. This config is " +
"deprecated and it will be removed in 3.0.0.")
.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,7 +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] = executedPlan.execute()
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
@@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution

import org.apache.spark.{Partition, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.internal.SQLConf

/**
* It is just a wrapper over `sqlRDD`, which sets and makes effective all the configs from the
* captured `SQLConf`.
* Please notice that this means we may miss configurations set after the creation of this RDD and
* before its execution.
*
* @param sqlRDD the `RDD` generated by the SQL plan
* @param conf the `SQLConf` to apply to the execution of the SQL plan
*/
class SQLExecutionRDD(
var sqlRDD: RDD[InternalRow], conf: SQLConf) extends RDD[InternalRow](sqlRDD) {
private val sqlConfigs = conf.getAllConfs
private lazy val sqlConfExecutorSide = {
val newConf = new SQLConf()
sqlConfigs.foreach { case (k, v) => newConf.setConfString(k, v) }
newConf
}

override val partitioner = firstParent[InternalRow].partitioner

override def getPartitions: Array[Partition] = firstParent[InternalRow].partitions

override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
// If we are in the context of a tracked SQL operation, `SQLExecution.EXECUTION_ID_KEY` is set
// and we have nothing to do here. Otherwise, we use the `SQLConf` captured at the creation of
// this RDD.
if (context.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) == null) {
SQLConf.withExistingConf(sqlConfExecutorSide) {
firstParent[InternalRow].iterator(split, context)
}
} else {
firstParent[InternalRow].iterator(split, context)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,13 @@

package org.apache.spark.sql.internal

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.{SparkException, SparkFunSuite}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlan}
import org.apache.spark.sql.execution.debug.codegenStringSeq
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.test.SQLTestUtils
Expand Down Expand Up @@ -98,4 +103,53 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils {
}
}
}

test("SPARK-28939: propagate SQLConf also in conversions to RDD") {
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])
}
}
}
}

case class SQLConfAssertPlan(confToCheck: Seq[(String, String)]) extends LeafExecNode {
override protected def doExecute(): RDD[InternalRow] = {
sqlContext
.sparkContext
.parallelize(0 until 2, 2)
.mapPartitions { it =>
val confs = SQLConf.get
confToCheck.foreach { case (key, expectedValue) =>
assert(confs.getConfString(key) == expectedValue)
}
it.map(i => InternalRow.fromSeq(Seq(i)))
}
}

override def output: Seq[Attribute] = Seq.empty
}

case class FakeQueryExecution(spark: SparkSession, physicalPlan: SparkPlan)
extends QueryExecution(spark, LocalRelation()) {
override lazy val sparkPlan: SparkPlan = physicalPlan
override lazy val executedPlan: SparkPlan = physicalPlan
}