Skip to content
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

[SPARK-27744][SQL] preserve spark properties on async subquery tasks #24625

Closed
wants to merge 1 commit 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
41 changes: 41 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Futures.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.util

import scala.concurrent.{ExecutionContext, Future}

import org.apache.spark.SparkContext


private[spark] object Futures {
def withLocalProperties[T](
sc: SparkContext)(
body: => T)(
implicit executor: ExecutionContext): Future[T] = {
val properties = sc.getLocalProperties
Future {
val originalProperties = sc.getLocalProperties
try {
sc.setLocalProperties(properties)
body
} finally {
sc.setLocalProperties(originalProperties)
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.types.{LongType, StructType}
import org.apache.spark.util.ThreadUtils
import org.apache.spark.util.{Futures, ThreadUtils}
import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler}

/** Physical plan for Project. */
Expand Down Expand Up @@ -696,12 +696,8 @@ case class SubqueryExec(name: String, child: SparkPlan)

@transient
private lazy val relationFuture: Future[Array[InternalRow]] = {
// relationFuture is used in "doExecute". Therefore we can get the execution id correctly here.
val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
Future {
// This will run in another thread. Set the execution id so that we can connect these jobs
// with the correct execution.
SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) {
Futures.withLocalProperties(sparkContext) {
SQLExecution.withSQLConfPropagated(sqlContext.sparkSession) {
val beforeCollect = System.nanoTime()
// Note that we use .executeCollect() because we don't want to convert data to Scala types
val rows: Array[InternalRow] = child.executeCollect()
Expand All @@ -710,7 +706,10 @@ case class SubqueryExec(name: String, child: SparkPlan)
val dataSize = rows.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum
longMetric("dataSize") += dataSize

SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq)
SQLMetrics.postDriverMetricUpdates(
sparkContext,
sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY),
metrics.values.toSeq)
rows
}
}(SubqueryExec.executionContext)
Expand All @@ -734,8 +733,9 @@ case class SubqueryExec(name: String, child: SparkPlan)
}

object SubqueryExec {
private[spark] val THREADS = 16
private[execution] val executionContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("subquery", 16))
ThreadUtils.newDaemonCachedThreadPool("subquery", THREADS))
}

/**
Expand Down
26 changes: 24 additions & 2 deletions sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,11 @@ package org.apache.spark.sql

import scala.collection.mutable.ArrayBuffer

import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, GenericInternalRow, Literal, SubqueryExpression, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
import org.apache.spark.sql.execution.{ExecSubqueryExpression, FileSourceScanExec, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.{ExecSubqueryExpression, FileSourceScanExec, ReusedSubqueryExec, ScalarSubquery, SparkPlan, SparkPlanTest, SubqueryExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.FileScanRDD
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
Expand Down Expand Up @@ -1383,4 +1385,24 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
assert(subqueryExecs.forall(_.name.startsWith("scalar-subquery#")),
"SubqueryExec name should start with scalar-subquery#")
}

test("SPARK-27744: Subquery execution preserves spark local properties") {
case class LocalPropertiesOperator(key: String, value: String) extends SparkPlan {
override protected def doExecute(): RDD[InternalRow] = {
assert(spark.sparkContext.getLocalProperty(key) == value)
sparkContext.parallelize(Seq(new UnsafeRow()))

}
override def output: Seq[Attribute] = Seq()
override def producedAttributes: AttributeSet = outputSet
override def children: Seq[SparkPlan] = Nil
}

spark.sparkContext.setLocalProperty("a", "1")
for (i <- 0 to SubqueryExec.THREADS) {
SubqueryExec("test", LocalPropertiesOperator("a", "1")).executeCollect()
}
spark.sparkContext.setLocalProperty("a", "2")
SubqueryExec("test", LocalPropertiesOperator("a", "2")).executeCollect()
}
}