Skip to content

Commit

Permalink
[SPARK-29233][K8S] Add regex expression checks for executorEnv…
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

In kubernetes, there are some naming regular expression requirements and restrictions on environment variable names, such as:

- In kubernetes version release-1.7 and earlier, the naming rules of pod environment variable names should meet the requirements of regular expressions: [[A-Za-z_] [A-Za-z0-9_]*](https://github.com/kubernetes/kubernetes/blob/release-1.7/staging/src/k8s.io/apimachinery/pkg/util/validation/validation.go#L169)
- In kubernetes version release-1.8 and later, the naming rules of pod environment variable names should meet the requirements of regular expressions: [[-. _ A-ZA-Z][-. _ A-ZA-Z0-9].*](https://github.com/kubernetes/kubernetes/blob/release-1.8/staging/src/k8s.io/apimachinery/pkg/util/validation/validation.go#L305)

However, in spark on k8s mode, spark should add restrictions on environmental variable names when creating executorEnv.

In addition, we need to use regular expressions adapted to the high version of k8s to increase the restrictions on the names of environmental variables.

Otherwise, the pod will not be created properly and the spark application will be suspended.

To solve the problem above, a regular validation to executorEnv is added and committed. 

### Why are the changes needed?

If no validation rules are added, the environment variable names that don't meet the requirements will cause the pod to not be created properly and the application will be suspended.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Add unit tests and manually run.

Closes #25920 from merrily01/SPARK-29233.

Authored-by: maruilei <maruilei@jd.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
  • Loading branch information
merrily01 authored and srowen committed Oct 6, 2019
1 parent 7c5db45 commit 77510c6
Show file tree
Hide file tree
Showing 2 changed files with 42 additions and 2 deletions.
Expand Up @@ -24,6 +24,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.deploy.k8s.Config._
import org.apache.spark.deploy.k8s.Constants._
import org.apache.spark.deploy.k8s.submit._
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.ConfigEntry
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -123,7 +124,7 @@ private[spark] class KubernetesExecutorConf(
val appId: String,
val executorId: String,
val driverPod: Option[Pod])
extends KubernetesConf(sparkConf) {
extends KubernetesConf(sparkConf) with Logging {

override val resourceNamePrefix: String = {
get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX).getOrElse(
Expand All @@ -148,7 +149,8 @@ private[spark] class KubernetesExecutorConf(
executorCustomLabels ++ presetLabels
}

override def environment: Map[String, String] = sparkConf.getExecutorEnv.toMap
override def environment: Map[String, String] = sparkConf.getExecutorEnv.filter(
p => checkExecutorEnvKey(p._1)).toMap

override def annotations: Map[String, String] = {
KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
Expand All @@ -166,6 +168,20 @@ private[spark] class KubernetesExecutorConf(
KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX)
}

private def checkExecutorEnvKey(key: String): Boolean = {
// Pattern for matching an executorEnv key, which meets certain naming rules.
val executorEnvRegex = "[-._a-zA-Z][-._a-zA-Z0-9]*".r
if (executorEnvRegex.pattern.matcher(key).matches()) {
true
} else {
logWarning(s"Invalid key: $key: " +
"a valid environment variable name must consist of alphabetic characters, " +
"digits, '_', '-', or '.', and must not start with a digit." +
s"Regex used for validation is '$executorEnvRegex')")
false
}
}

}

private[spark] object KubernetesConf {
Expand Down
Expand Up @@ -44,6 +44,12 @@ class KubernetesConfSuite extends SparkFunSuite {
"customEnvKey2" -> "customEnvValue2")
private val DRIVER_POD = new PodBuilder().build()
private val EXECUTOR_ID = "executor-id"
private val EXECUTOR_ENV_VARS = Map(
"spark.executorEnv.1executorEnvVars1/var1" -> "executorEnvVars1",
"spark.executorEnv.executorEnvVars2*var2" -> "executorEnvVars2",
"spark.executorEnv.executorEnvVars3_var3" -> "executorEnvVars3",
"spark.executorEnv.executorEnvVars4-var4" -> "executorEnvVars4",
"spark.executorEnv.executorEnvVars5-var5" -> "executorEnvVars5/var5")

test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") {
val sparkConf = new SparkConf(false)
Expand Down Expand Up @@ -132,4 +138,22 @@ class KubernetesConfSuite extends SparkFunSuite {
assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS)
assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS)
}

test("Verify that executorEnv key conforms to the regular specification") {
val sparkConf = new SparkConf(false)
EXECUTOR_ENV_VARS.foreach { case (key, value) =>
sparkConf.set(key, value)
}

val conf = KubernetesConf.createExecutorConf(
sparkConf,
EXECUTOR_ID,
KubernetesTestConf.APP_ID,
Some(DRIVER_POD))
assert(conf.environment ===
Map(
"executorEnvVars3_var3" -> "executorEnvVars3",
"executorEnvVars4-var4" -> "executorEnvVars4",
"executorEnvVars5-var5" -> "executorEnvVars5/var5"))
}
}

0 comments on commit 77510c6

Please sign in to comment.