Skip to content

Commit

Permalink
[SPARK-33720][K8S] Support submit to k8s only with token
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

Support submit to k8s only with token.

### Why are the changes needed?

Now, sumbit to k8s always need oauth files.

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

### How was this patch tested?

Before, submit job out of k8s cluster without correct ca.crt, we may get this exception:
```
Caused by: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
        at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:439)
        at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:306)
        at sun.security.validator.Validator.validate(Validator.java:271)
        at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:312)
```
When set spark.kubernetes.trust.certificates = true, we can submit only with correct token, no need to config ca.crt in local env.
Submit as:
```
 bin/spark-submit \
     --master $master \
     --name pi \
     --deploy-mode cluster \
     --conf spark.kubernetes.container.image=$image \
     --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \
     --conf spark.kubernetes.authenticate.submission.oauthToken=$clusterToken \
     --conf spark.kubernetes.trust.certificates=true \
     local:///opt/spark/examples/src/main/python/pi.py 200
```

Closes apache#30684 from hddong/trust-certs.

Authored-by: hongdongdong <hongdongdong@cmss.chinamobile.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
  • Loading branch information
hddong authored and dongjoon-hyun committed Mar 24, 2021
1 parent 0494dc9 commit 985c653
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 0 deletions.
Expand Up @@ -443,6 +443,13 @@ private[spark] object Config extends Logging {
val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX =
"spark.kubernetes.authenticate.submission"

val KUBERNETES_TRUST_CERTIFICATES =
ConfigBuilder("spark.kubernetes.trust.certificates")
.doc("If set to true then client can submit to kubernetes cluster only with token")
.version("3.2.0")
.booleanConf
.createWithDefault(false)

val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector."

val KUBERNETES_DELETE_EXECUTORS =
Expand Down
Expand Up @@ -83,6 +83,7 @@ private[spark] object SparkKubernetesClientFactory extends Logging {
.withWebsocketPingInterval(0)
.withRequestTimeout(clientType.requestTimeout(sparkConf))
.withConnectionTimeout(clientType.connectionTimeout(sparkConf))
.withTrustCerts(sparkConf.get(KUBERNETES_TRUST_CERTIFICATES))
.withOption(oauthTokenValue) {
(token, configBuilder) => configBuilder.withOauthToken(token)
}.withOption(oauthTokenFile) {
Expand Down

0 comments on commit 985c653

Please sign in to comment.