This is an automated email from the ASF dual-hosted git repository. srowen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 43bf4ae [SPARK-26914][SQL] Fix scheduler pool may be unpredictable when we only want to use default pool and do not set spark.scheduler.pool for the session 43bf4ae is described below commit 43bf4ae6417fcb15d0fbc7880f14f307c164d464 Author: zhoukang <zhoukang199...@gmail.com> AuthorDate: Thu Mar 28 09:24:16 2019 -0500 [SPARK-26914][SQL] Fix scheduler pool may be unpredictable when we only want to use default pool and do not set spark.scheduler.pool for the session ## What changes were proposed in this pull request? When using fair scheduler mode for thrift server, we may have unpredictable result. ``` val pool = sessionToActivePool.get(parentSession.getSessionHandle) if (pool != null) { sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) } ``` The cause is we use thread pool to execute queries for thriftserver, and when we call setLocalProperty we may have unpredictab behavior. ``` /** * Set a local property that affects jobs submitted from this thread, such as the Spark fair * scheduler pool. User-defined properties may also be set here. These properties are propagated * through to worker tasks and can be accessed there via * [[org.apache.spark.TaskContext#getLocalProperty]]. * * These properties are inherited by child threads spawned from this thread. This * may have unexpected consequences when working with thread pools. The standard java * implementation of thread pools have worker threads spawn other worker threads. * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String) { if (value == null) { localProperties.get.remove(key) } else { localProperties.get.setProperty(key, value) } } ``` I post an example on https://jira.apache.org/jira/browse/SPARK-26914 . ## How was this patch tested? UT Closes #23826 from caneGuy/zhoukang/fix-scheduler-error. Authored-by: zhoukang <zhoukang199...@gmail.com> Signed-off-by: Sean Owen <sean.o...@databricks.com> --- .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 1772fe6..b05307e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -226,9 +226,9 @@ private[hive] class SparkExecuteStatementOperation( parentSession.getUsername) sqlContext.sparkContext.setJobGroup(statementId, statement) val pool = sessionToActivePool.get(parentSession.getSessionHandle) - if (pool != null) { - sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) - } + // It may have unpredictably behavior since we use thread pools to execute quries and + // the 'spark.scheduler.pool' may not be 'default' when we did not set its value.(SPARK-26914) + sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) try { result = sqlContext.sql(statement) logDebug(result.queryExecution.toString()) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org