diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index c7a008ab11282..8ce475a680319 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -496,7 +496,8 @@ class SparkSession private( parsedPlan } } - Dataset.ofRows(self, plan, tracker) + Dataset.ofRows(self, plan, tracker, + QueryExecution.determineShuffleCleanupMode(sessionState.conf)) } /** @inheritdoc */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index de0fde16e5d03..c969e58c78dce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -67,6 +67,18 @@ class AdaptiveQueryExecSuite setupTestData() + protected override def beforeAll(): Unit = { + super.beforeAll() + // Tests depend on intermediate results that would otherwise be cleaned up when + // shuffle clean up is enabled, causing test failures. + sqlConf.setConf(SQLConf.CLASSIC_SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED, false) + } + + protected override def afterAll(): Unit = { + sqlConf.setConf(SQLConf.CLASSIC_SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED, true) + super.afterAll() + } + private def runAdaptiveAndVerifyResult(query: String, skipCheckAnswer: Boolean = false): (SparkPlan, SparkPlan) = { var finalPlanCnt = 0 @@ -1871,9 +1883,7 @@ class AdaptiveQueryExecSuite } withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.SHUFFLE_PARTITIONS.key -> "5", - // Disabling cleanup as the test assertions depend on them - SQLConf.CLASSIC_SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED.key -> "false") { + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { val df = sql( """ |SELECT * FROM ( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 8b9b7352fdca2..3c904b6e3a349 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -30,8 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.COMMAND import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.CommandResult -import org.apache.spark.sql.classic.ClassicConversions._ -import org.apache.spark.sql.execution.{QueryExecution, QueryExecutionException, SQLExecution} + import org.apache.spark.sql.execution.{QueryExecution, QueryExecutionException, SQLExecution} import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution} import org.apache.spark.util.Utils @@ -67,7 +66,7 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv. new VariableSubstitution().substitute(command) } sparkSession.sparkContext.setJobDescription(substitutorCommand) - val execution = sparkSession.sessionState.executePlan(sparkSession.sql(command).logicalPlan) + val execution = sparkSession.sql(command).queryExecution // The SQL command has been executed above via `executePlan`, therefore we don't need to // wrap it again with a new execution ID when getting Hive result. execution.logical match {