dongjoon-hyun commented on code in PR #35886:
URL: https://github.com/apache/spark/pull/35886#discussion_r945181441


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala:
##########
@@ -122,68 +122,41 @@ private[spark] class BasicExecutorFeatureStep(
       
buildExecutorResourcesQuantities(execResources.customResources.values.toSet)
 
     val executorEnv: Seq[EnvVar] = {
-        (Seq(
-          (ENV_DRIVER_URL, driverUrl),
-          (ENV_EXECUTOR_CORES, execResources.cores.get.toString),
-          (ENV_EXECUTOR_MEMORY, executorMemoryString),
-          (ENV_APPLICATION_ID, kubernetesConf.appId),
-          // This is to set the SPARK_CONF_DIR to be /opt/spark/conf
-          (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL),
-          (ENV_EXECUTOR_ID, kubernetesConf.executorId),
-          (ENV_RESOURCE_PROFILE_ID, resourceProfile.id.toString)
-        ) ++ kubernetesConf.environment).map { case (k, v) =>
-          new EnvVarBuilder()
-            .withName(k)
-            .withValue(v)
-            .build()
-        }
-      } ++ {
-        Seq(new EnvVarBuilder()
-          .withName(ENV_EXECUTOR_POD_IP)
-          .withValueFrom(new EnvVarSourceBuilder()
-            .withNewFieldRef("v1", "status.podIP")
-            .build())
-          .build())
-      } ++ {
-        Seq(new EnvVarBuilder()
-          .withName(ENV_EXECUTOR_POD_NAME)
-          .withValueFrom(new EnvVarSourceBuilder()
-            .withNewFieldRef("v1", "metadata.name")
-            .build())
-          .build())
-      } ++ {
-        if (kubernetesConf.get(AUTH_SECRET_FILE_EXECUTOR).isEmpty) {
-          Option(secMgr.getSecretKey()).map { authSecret =>
-            new EnvVarBuilder()
-              .withName(SecurityManager.ENV_AUTH_SECRET)
-              .withValue(authSecret)
-              .build()
-          }
-        } else None
-      } ++ {
-        kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp =>
-          new EnvVarBuilder()
-            .withName(ENV_CLASSPATH)
-            .withValue(cp)
-            .build()
-        }
-      } ++ {
-        val userOpts = kubernetesConf.get(EXECUTOR_JAVA_OPTIONS).toSeq.flatMap 
{ opts =>
-          val subsOpts = Utils.substituteAppNExecIds(opts, 
kubernetesConf.appId,
-            kubernetesConf.executorId)
+      val sparkAuthSecret = if 
(kubernetesConf.get(AUTH_SECRET_FILE_EXECUTOR).isEmpty) {
+        Map(SecurityManager.ENV_AUTH_SECRET -> secMgr.getSecretKey())
+      } else {
+        Nil
+      }
+      val userOpts = kubernetesConf.get(EXECUTOR_JAVA_OPTIONS).toSeq.flatMap { 
opts =>
+        val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId,
+          kubernetesConf.executorId)
           Utils.splitCommandString(subsOpts)
-        }
-
-        val sparkOpts = Utils.sparkJavaOpts(kubernetesConf.sparkConf,
-          SparkConf.isExecutorStartupConf)
-
-        (userOpts ++ sparkOpts).zipWithIndex.map { case (opt, index) =>
-          new EnvVarBuilder()
-            .withName(s"$ENV_JAVA_OPT_PREFIX$index")
-            .withValue(opt)
-            .build()
-        }
       }
+      val sparkOpts = Utils.sparkJavaOpts(kubernetesConf.sparkConf,
+        SparkConf.isExecutorStartupConf)
+      val allOpts = (userOpts ++ sparkOpts).zipWithIndex.map { case (opt, 
index) =>
+        (s"$ENV_JAVA_OPT_PREFIX$index", opt)
+      }.toMap
+      KubernetesUtils.buildEnvVars(
+        Map(
+          ENV_DRIVER_URL -> driverUrl,
+          ENV_EXECUTOR_CORES -> execResources.cores.get.toString,
+          ENV_EXECUTOR_MEMORY -> executorMemoryString,
+          ENV_APPLICATION_ID -> kubernetesConf.appId,
+          // This is to set the SPARK_CONF_DIR to be /opt/spark/conf
+          ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL,
+          ENV_EXECUTOR_ID -> kubernetesConf.executorId,
+          ENV_RESOURCE_PROFILE_ID -> resourceProfile.id.toString)
+          ++ kubernetesConf.environment
+          ++ sparkAuthSecret
+          ++ Map(ENV_CLASSPATH-> 
kubernetesConf.get(EXECUTOR_CLASS_PATH).orNull)
+          ++ allOpts) ++
+      KubernetesUtils.buildEnvVarsWithFieldRef(
+        Seq(
+          (ENV_EXECUTOR_POD_IP, "v1", "status.podIP"),
+          (ENV_EXECUTOR_POD_NAME, "v1", "metadata.name")
+        ))

Review Comment:
   Technically, new logic looks inconsistent to me in terms of the order of 
variable definition.
   New logic seems to have a limitation where `buildEnvVars`-defined ones are 
overwritten by `buildEnvVarsWithFieldRef`-defined ones always. So, do we assume 
variable uniqueness here?



-- 
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