Skip to content

Conversation

@dcoliversun
Copy link
Contributor

@dcoliversun dcoliversun commented Mar 17, 2022

What changes were proposed in this pull request?

This PR introduces buildEnvVars and buildEnvVarsWithFieldRef functions for kubernetesUtils to eliminate duplicate code pattern.

Why are the changes needed?

Code simplification. Remove redundant EnvVar Build code for driver and executor pod.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

  • Add unit test and Pass the GA.
  • Running k8s integration test manually:
$ build/sbt -Pkubernetes -Pkubernetes-integration-tests -Dtest.exclude.tags=minikube,r "kubernetes-integration-tests/test"

[info] KubernetesSuite:
[info] - Run SparkPi with no resources (18 seconds, 285 milliseconds)
[info] - Run SparkPi with no resources & statefulset allocation (18 seconds, 56 milliseconds)
[info] - Run SparkPi with a very long application name. (15 seconds, 828 milliseconds)
[info] - Use SparkLauncher.NO_RESOURCE (13 seconds, 807 milliseconds)
[info] - Run SparkPi with a master URL without a scheme. (14 seconds, 504 milliseconds)
[info] - Run SparkPi with an argument. (13 seconds, 573 milliseconds)
[info] - Run SparkPi with custom labels, annotations, and environment variables. (17 seconds, 107 milliseconds)
[info] - All pods have the same service account by default (12 seconds, 551 milliseconds)
[info] - Run extraJVMOptions check on driver (7 seconds, 452 milliseconds)
[info] - Run SparkRemoteFileTest using a remote data file (13 seconds, 910 milliseconds)
[info] - Verify logging configuration is picked from the provided SPARK_CONF_DIR/log4j2.properties (26 seconds, 493 milliseconds)
[info] - Run SparkPi with env and mount secrets. (24 seconds, 546 milliseconds)
[info] - Run PySpark on simple pi.py example (14 seconds, 609 milliseconds)
[info] - Run PySpark to test a pyfiles example (17 seconds, 710 milliseconds)
[info] - Run PySpark with memory customization (14 seconds, 918 milliseconds)
[info] - Run in client mode. (10 seconds, 385 milliseconds)
[info] - Start pod creation from template (14 seconds, 522 milliseconds)
[info] - SPARK-38398: Schedule pod creation from template (12 seconds, 771 milliseconds)
[info] - Test basic decommissioning (48 seconds, 155 milliseconds)
[info] - Test basic decommissioning with shuffle cleanup (47 seconds, 723 milliseconds)
[info] - Test decommissioning with dynamic allocation & shuffle cleanups (2 minutes, 48 seconds)
[info] - Test decommissioning timeouts (47 seconds, 866 milliseconds)
[info] - SPARK-37576: Rolling decommissioning (1 minute, 45 seconds)
[info] Run completed in 13 minutes, 51 seconds.
[info] Total number of tests run: 23
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 23, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.

…ieldRef for KubernetesUtils to eliminate duplicate code pattern
@dcoliversun
Copy link
Contributor Author

@dongjoon-hyun
It would be good if you could take a look when you have time, thanks!

Copy link
Member

@Yikun Yikun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In general, I think refactoring helps to simplify the code. +1 on the idea. But for core featurestep changes we need to make sure test coverage all cases.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@dcoliversun
Copy link
Contributor Author

@Yikun @martin-g
Update code for comments. It's good if you have a look again.

Copy link
Member

@martin-g martin-g left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!
The suggestions below are just nits which you could ignore!

@dcoliversun dcoliversun changed the title [SPARK-38582][K8S] Introduce buildEnvVarsWithKV and buildEnvVarsWithFieldRef for KubernetesUtils to eliminate duplicate code pattern [SPARK-38582][K8S] Introduce buildEnvVars and buildEnvVarsWithFieldRef for KubernetesUtils to eliminate duplicate code pattern Mar 18, 2022
@dcoliversun
Copy link
Contributor Author

dcoliversun commented Mar 21, 2022

Gentle ping @dongjoon-hyun @martin-g
Could this PR be merged?

@martin-g
Copy link
Member

I am just a contributor to the project. A project member could merge PRs.
// CC @dongjoon-hyun @HyukjinKwon @holdenk

@dongjoon-hyun
Copy link
Member

Sorry for being late. To prepare Apache Spark 3.3.0, I was too busy in these days.

}

val driverCustomEnvs = KubernetesUtils.buildEnvVars(
conf.environment + (ENV_APPLICATION_ID -> conf.appId))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ur, are you sure about this change? This looks inconsistent to me because this breaks the existing behavior. Previously, conf.environment's ENV_APPLICATION_ID supersedes if there is a duplication.

Copy link
Contributor Author

@dcoliversun dcoliversun Apr 6, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your worries are right. After the latest commit, I don't think this refactoring will introduce lots of regressions.

  1. When there are two environment variables with the same key in the pod, the last value shall work.

I apply a pod with yaml

apiVersion: v1
kind: Pod
metadata:
  name: static-web
  labels:
    role: myrole
spec:
  containers:
    - name: web
      image: nginx
      env:
      - name: DEMO_GREETING
        value: "Hello from the environment"
      - name: DEMO_GREETING
        value: "Such a sweet sorrow"
      ports:
        - name: web
          containerPort: 80
          protocol: TCP

And execute the echo command, get the value of DEMO_GREETING is Such a sweet sorrow.

$ kubectl exec -ti static-web -n default -- /bin/bash
$ echo $DEMO_GREETING
Such a sweet sorrow
  1. ++ operation in Scala Map is overwrite append operation. As long as the order of our ++ remains the same as the previous env order, the value of the environment variable in the pod will not change. For example, user specified ENV_APPLICATION_ID as temp_app_id in sparkConf
spark.kubernetes.driverEnv.ENV_APPLICATION_ID=user_specified_app_id

According to the previous logic, two envs with the same key will be generated

env:
  - name: ENV_APPLICATION_ID
    value: spark-XXXXXXX # https://github.com/apache/spark/blob/b852645f69b3b7a0a2140a732c4c03b302f8795a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala#L222
  - name: ENV_APPLICATION_ID
    value: user_specified_app_id

According to the first point, the value of ENV_APPLICATION_ID is user_specified_app_id.

We replace seq with map. It causes the env as follow

env:
  - name: ENV_APPLICATION_ID
    value: user_specified_app_id

Because ++ in map is overwrite operation, so the value of ENV_APPLICATION_ID is overwritten. Finally, the value of ENV_APPLICATION_ID in pod is user_specified_app_id.

I agree that this refactoring does not bring env changes and can reduce unnecessary env.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for the intention, but the proposed PR looks a little intrusive and inconsistent to me.

This PR may introduce lots of regression if we are not careful enough to avoid any regression like the following comment.

@dongjoon-hyun
Copy link
Member

FYI, we had better hold on these kind of PRs during the planned release process. It's the same for the other refactoring PRs.

@dcoliversun
Copy link
Contributor Author

@dongjoon-hyun Hi, I'm ok to hold on this PR during the planned release process. Thanks for your work on the release plan. And I'm addressing your comment.

@dongjoon-hyun
Copy link
Member

Thank you, @dcoliversun !

@dcoliversun
Copy link
Contributor Author

+1 for the intention, but the proposed PR looks a little intrusive and inconsistent to me.

This PR may introduce lots of regression if we are not careful enough to avoid any regression like the following comment.

@dongjoon-hyun Hi, reply about this worry is here
#35886 (comment)

@dongjoon-hyun
Copy link
Member

Thank you for your reply, @dcoliversun .

.orElse(environmentVariables.get(ENV_PYSPARK_DRIVER_PYTHON))
.orElse(environmentVariables.get(ENV_PYSPARK_PYTHON))
.orNull))
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you check what happens when both PYSPARK_PYTHON and ENV_PYSPARK_PYTHON is not defined? New logic looks not identical with the previous one.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let me check it

Copy link
Contributor Author

@dcoliversun dcoliversun Aug 15, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe new logic is identical with previous one.

Old logic

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

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

...
spec:
  ...
  env:
  - name: PYSPARK_PYTHON
  - name: PYSPARK_DRIVER_PYTHON

Exec echo command, the result is as follows.

$ echo $PYSPARK_PYTHON
<BLANK LINE>
$ echo $PYSPARK_DRIVER_PYTHON
<BLANK LINE>

New logic

When both PYSPARK_PYTHON and ENV_PYSPARK_PYTHON is not defined, no envVars are set in Pod. It behaves the same as environment variable with NULL value.

Existing code can also illustrate these, unit test about auth secret propagation expects that SPARK_CLASSPATH doesn't exists in executor environment variables.

test("auth secret propagation") {
val conf = baseConf.clone()
.set(config.NETWORK_AUTH_ENABLED, true)
.set("spark.master", "k8s://127.0.0.1")
val secMgr = new SecurityManager(conf)
secMgr.initializeAuth()
val step = new BasicExecutorFeatureStep(KubernetesTestConf.createExecutorConf(sparkConf = conf),
secMgr, defaultProfile)
val executor = step.configurePod(SparkPod.initialPod())
checkEnv(executor, conf, Map(SecurityManager.ENV_AUTH_SECRET -> secMgr.getSecretKey()))
}

// Check that the expected environment variables are present.
private def checkEnv(
executorPod: SparkPod,
conf: SparkConf,
additionalEnvVars: Map[String, String]): Unit = {
val defaultEnvs = Map(
ENV_EXECUTOR_ID -> "1",
ENV_DRIVER_URL -> DRIVER_ADDRESS.toString,
ENV_EXECUTOR_CORES -> "1",
ENV_EXECUTOR_MEMORY -> "1024m",
ENV_APPLICATION_ID -> KubernetesTestConf.APP_ID,
ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL,
ENV_SPARK_USER -> Utils.getCurrentUserName(),
ENV_RESOURCE_PROFILE_ID -> "0",
// These are populated by K8s on scheduling
ENV_EXECUTOR_POD_IP -> null,
ENV_EXECUTOR_POD_NAME -> null)
val extraJavaOptsStart = additionalEnvVars.keys.count(_.startsWith(ENV_JAVA_OPT_PREFIX))
val extraJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf)
val extraJavaOptsEnvs = extraJavaOpts.zipWithIndex.map { case (opt, ind) =>
s"$ENV_JAVA_OPT_PREFIX${ind + extraJavaOptsStart}" -> opt
}.toMap
val containerEnvs = executorPod.container.getEnv.asScala.map {
x => (x.getName, x.getValue)
}.toMap
val expectedEnvs = defaultEnvs ++ additionalEnvVars ++ extraJavaOptsEnvs
assert(containerEnvs === expectedEnvs)
}

} ++ {
kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp =>
new EnvVarBuilder()
.withName(ENV_CLASSPATH)
.withValue(cp)
.build()
}
} ++ {

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for in-depth analysis.

Seq(
(ENV_EXECUTOR_POD_IP, "v1", "status.podIP"),
(ENV_EXECUTOR_POD_NAME, "v1", "metadata.name")
))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need variable uniqueness. As you said, envVar is overwrite in pod.spec.container.envVar.

  • buildEnvVarsWithFieldRef method can project Pod-level fields into the running container as environment variables, which value is defined until pod scheduled or later.
  • buildEnvVars method only put prepared key-value into pre-submission container as environment variables.
    From normal usage, I believe buildEnvVarsWithFieldRef has higher priority than buildEnvVars.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ack, @dcoliversun .

}

/**
* This function builds the EnvVar objects for each key-value env.
Copy link
Member

@dongjoon-hyun dongjoon-hyun Aug 19, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The function description is inconsistent from the code body because we don't build EnvVar when value is null.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, in the function description, can we explicitly mention that we can use an empty string value to define a key-only EnvVar?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should mention it, because it's supported to set on yaml and works normally in container.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I made a PR to you, @dcoliversun .

@dcoliversun
Copy link
Contributor Author

@dongjoon-hyun Thanks for you PR and advice :) I'm working on it. Update later

dcoliversun and others added 3 commits August 19, 2022 16:52
address comments

address comments

address comments
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @dcoliversun .
I fixed a few typos at the last commit.
Merged to master for Apache Spark 3.4.0.

@dongjoon-hyun
Copy link
Member

Also, thank you, @Yikun and @martin-g , too.

@dcoliversun
Copy link
Contributor Author

Thanks for your help @dongjoon-hyun

@dcoliversun dcoliversun deleted the SPARK-38582 branch August 19, 2022 21:06
@dongjoon-hyun dongjoon-hyun removed their assignment Apr 14, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants