dcoliversun commented on code in PR #35886:
URL: https://github.com/apache/spark/pull/35886#discussion_r945429105


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala:
##########
@@ -84,25 +84,18 @@ private[spark] class DriverCommandFeatureStep(conf: 
KubernetesDriverConf)
           "variables instead.")
     }
 
-    val pythonEnvs =
-      Seq(
-        conf.get(PYSPARK_PYTHON)
-          .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON)).map { value =>
-          new EnvVarBuilder()
-            .withName(ENV_PYSPARK_PYTHON)
-            .withValue(value)
-            .build()
-        },
-        conf.get(PYSPARK_DRIVER_PYTHON)
-          .orElse(conf.get(PYSPARK_PYTHON))
-          .orElse(environmentVariables.get(ENV_PYSPARK_DRIVER_PYTHON))
-          .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON)).map { value =>
-          new EnvVarBuilder()
-            .withName(ENV_PYSPARK_DRIVER_PYTHON)
-            .withValue(value)
-            .build()
-        }
-      ).flatten
+    val pythonEnvs = {
+      KubernetesUtils.buildEnvVars(
+        Map(
+          ENV_PYSPARK_PYTHON -> conf.get(PYSPARK_PYTHON)
+            .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON))
+            .orNull,
+          ENV_PYSPARK_DRIVER_PYTHON -> conf.get(PYSPARK_DRIVER_PYTHON)
+            .orElse(conf.get(PYSPARK_PYTHON))
+            .orElse(environmentVariables.get(ENV_PYSPARK_DRIVER_PYTHON))
+            .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON))
+            .orNull))
+    }

Review Comment:
   Yes, before 
[d2395f0](https://github.com/apache/spark/pull/35886/commits/d2395f0ea33151243c43faebb2894a7585608d3b),
 new logic is not identical.  
   ### Old logic
   
https://github.com/apache/spark/blob/36dd531a93af55ce5c2bfd8d275814ccb2846962/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala#L87-L105
   In old logic, when both `PYSPARK_PYTHON` and `ENV_PYSPARK_PYTHON` is not 
defined, spark pod has empty environment variables, which of keys are 
`ENV_PYSPARK_PYTHON` and `ENV_PYSPARK_DRIVER_PYTHON`, and values are `NULL`.  
The corresponding Pod yaml is as follows  
   ```yaml
   ...
   spec:
     ...
     env:
     - name: PYSPARK_PYTHON
     - name: PYSPARK_DRIVER_PYTHON
   ```
   Exec `echo` command, the result is as follows. 
   ```shell
   $ echo $PYSPARK_PYTHON
   <BLANK LINE>
   $ echo $PYSPARK_DRIVER_PYTHON
   <BLANK LINE>
   ```
   ### New logic
   ```scala
     val pythonEnvs = {
         KubernetesUtils.buildEnvVars(
           Map(
             ENV_PYSPARK_PYTHON -> conf.get(PYSPARK_PYTHON)
               .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON))
               .orNull,
             ENV_PYSPARK_DRIVER_PYTHON -> conf.get(PYSPARK_DRIVER_PYTHON)
               .orElse(conf.get(PYSPARK_PYTHON))
               .orElse(environmentVariables.get(ENV_PYSPARK_DRIVER_PYTHON))
               .orElse(environmentVariables.get(ENV_PYSPARK_PYTHON))
               .orNull))
       }
   
     /**
      * This function builds the EnvVar objects for each key-value env.
      */
     @Since("3.4.0")
     def buildEnvVars(env: Map[String, String]): Seq[EnvVar] = {
       env.map { env =>
         new EnvVarBuilder()
           .withName(env._1)
           .withValue(env._2)
           .build()
       }.toSeq
     }
   
     /**
      * This function builds the EnvVar objects for each field ref env.
      */
     @Since("3.4.0")
     def buildEnvVarsWithFieldRef(env: Seq[(String, String, String)]): 
Seq[EnvVar] = {
       env.map { env =>
         new EnvVarBuilder()
           .withName(env._1)
           .withValueFrom(new EnvVarSourceBuilder()
             .withNewFieldRef(env._2, env._3)
             .build())
           .build()
       }
     }
   ```
   Now, 
[d2395f0](https://github.com/apache/spark/pull/35886/commits/d2395f0ea33151243c43faebb2894a7585608d3b)
 allows to set empty environment variables on pod. When second value of `env` 
is null in `buildEnvVars` method, it still used in `EnvVarBuilder` method.
   ```scala
   > print(env)
   Some(EnvVar(name=PYSPARK_DRIVER_PYTHON, value=null, valueFrom=null, 
additionalProperties={}))
   ```
   I believe 
[d2395f0](https://github.com/apache/spark/pull/35886/commits/d2395f0ea33151243c43faebb2894a7585608d3b)
 can keep identical with the previous one.  
   Unit test about case that both `PYSPARK_PYTHON` and `ENV_PYSPARK_PYTHON` is 
not defined is added.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to