Github user gengliangwang commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21299#discussion_r189204217
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala ---
    @@ -90,13 +92,42 @@ object SQLExecution {
        * thread from the original one, this method can be used to connect the 
Spark jobs in this action
        * with the known executionId, e.g., 
`BroadcastExchangeExec.relationFuture`.
        */
    -  def withExecutionId[T](sc: SparkContext, executionId: String)(body: => 
T): T = {
    +  def withExecutionId[T](sparkSession: SparkSession, executionId: 
String)(body: => T): T = {
    +    val sc = sparkSession.sparkContext
         val oldExecutionId = sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
    +    withSQLConfPropagated(sparkSession) {
    +      try {
    +        sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, executionId)
    +        body
    +      } finally {
    +        sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, oldExecutionId)
    +      }
    +    }
    +  }
    +
    +  def withSQLConfPropagated[T](sparkSession: SparkSession)(body: => T): T 
= {
    --- End diff --
    
    Maybe cleaner with following way:
    ```
      def withSQLConfPropagated[T](sparkSession: SparkSession)(body: => T): T = 
{
        val sc = sparkSession.sparkContext
        // Set all the specified SQL configs to local properties, so that they 
can be available at
        // the executor side.
        val allConfigs = sparkSession.sessionState.conf.getAllConfs
        val originalLocalProps = allConfigs.collect {
          case (key, value) if key.startsWith("spark") =>
            val originalValue = sc.getLocalProperty(key)
            sc.setLocalProperty(key, value)
            (key, originalValue)
        }
    
        try {
          body
        } finally {
          originalLocalProps.foreach {
            case (key, value) => sc.setLocalProperty(key, value)
          }
        }
      }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to