Commit graph

75 commits

Author SHA1 Message Date
Marcelo Vanzin 14f714fb30 [SPARK-26420][K8S] Generate more unique IDs when creating k8s resource names.
Using the current time as an ID is more prone to clashes than people generally
realize, so try to make things a bit more unique without necessarily using a
UUID, which would eat too much space in the names otherwise.

The implemented approach uses some bits from the current time, plus some random
bits, which should be more resistant to clashes.

Closes #23805 from vanzin/SPARK-26420.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-02-28 20:39:13 -08:00
Marcelo Vanzin a6ddc9d083 [SPARK-24736][K8S] Let spark-submit handle dependency resolution.
Before this change, there was some code in the k8s backend to deal
with how to resolve dependencies and make them available to the
Spark application. It turns out that none of that code is necessary,
since spark-submit already handles all that for applications started
in client mode - like the k8s driver that is run inside a Spark-created
pod.

For that reason, specifically for pyspark, there's no need for the
k8s backend to deal with PYTHONPATH; or, in general, to change the URIs
provided by the user at all. spark-submit takes care of that.

For testing, I created a pyspark script that depends on another module
that is shipped with --py-files. Then I used:

- --py-files http://.../dep.py http://.../test.py
- --py-files http://.../dep.zip http://.../test.py
- --py-files local:/.../dep.py local:/.../test.py
- --py-files local:/.../dep.zip local:/.../test.py

Without this change, all of the above commands fail. With the change, the
driver is able to see the dependencies in all the above cases; but executors
don't see the dependencies in the last two. That's a bug in shared Spark code
that deals with local: dependencies in pyspark (SPARK-26934).

I also tested a Scala app using the main jar from an http server.

Closes #23793 from vanzin/SPARK-24736.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-27 09:49:31 -08:00
liuxian 7912dbb88f [MINOR] Simplify boolean expression
## What changes were proposed in this pull request?

Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)

## How was this patch tested?
N/A

Closes #23884 from 10110346/simplifyboolean.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-27 08:38:00 -06:00
Marcelo Vanzin afbff6446f Revert "[SPARK-26742][K8S] Update Kubernetes-Client version to 4.1.2"
This reverts commit a3192d966a.
2019-02-26 13:42:07 -08:00
Jiaxin Shan a3192d966a [SPARK-26742][K8S] Update Kubernetes-Client version to 4.1.2
## What changes were proposed in this pull request?
Changed the `kubernetes-client` version to 4.1.2.  Latest version fix error with exec credentials (used by aws eks) and this will be used to talk with kubernetes API server. Users can submit spark job to EKS api endpoint now with this patch.

## How was this patch tested?
unit tests and manual tests.

Closes #23814 from Jeffwan/update_k8s_sdk.

Authored-by: Jiaxin Shan <seedjeffwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-25 04:56:04 -06:00
Marcelo Vanzin 61c3cdc706 [SPARK-24894][K8S] Make sure valid host names are created for executors.
Since the host name is derived from the app name, which can contain arbitrary
characters, it needs to be sanitized so that only valid characters are allowed.

On top of that, take extra care that truncation doesn't leave characters that
are valid except at the start of a host name.

Closes #23781 from vanzin/SPARK-24894.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-19 15:19:59 -08:00
Marcelo Vanzin 2a67dbfbd3 [SPARK-26595][CORE] Allow credential renewal based on kerberos ticket cache.
This change addes a new mode for credential renewal that does not require
a keytab; it uses the local ticket cache instead, so it works while the
user keeps the cache valid.

This can be useful for, e.g., people running long spark-shell sessions where
their kerberos login is kept up-to-date.

The main change to enable this behavior is in HadoopDelegationTokenManager,
with a small change in the HDFS token provider. The other changes are to avoid
creating duplicate tokens when submitting the application to YARN; they allow
the tokens from the scheduler to be sent to the YARN AM, reducing the round trips
to HDFS.

For that, the scheduler initialization code was changed a little bit so that
the tokens are available when the YARN client is initialized. That basically
takes care of a long-standing TODO that was in the code to clean up configuration
propagation to the driver's RPC endpoint (in CoarseGrainedSchedulerBackend).

Tested with an app designed to stress this functionality, with both keytab and
cache-based logins. Some basic kerberos tests on k8s also.

Closes #23525 from vanzin/SPARK-26595.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-28 13:32:34 -08:00
Rob Vesse c542c247bb [SPARK-25887][K8S] Configurable K8S context support
This enhancement allows for specifying the desired context to use for the initial K8S client auto-configuration.  This allows users to more easily access alternative K8S contexts without having to first
explicitly change their current context via kubectl.

Explicitly set my K8S context to a context pointing to a non-existent cluster, then launched Spark jobs with explicitly specified contexts via the new `spark.kubernetes.context` configuration property.

Example Output:

```
> kubectl config current-context
minikube
> minikube status
minikube: Stopped
cluster:
kubectl:
> ./spark-submit --master k8s://https://localhost:6443 --deploy-mode cluster --name spark-pi --class org.apache.spark.examples.SparkPi --conf spark.executor.instances=2 --conf spark.kubernetes.context=docker-for-desktop --conf spark.kubernetes.container.image=rvesse/spark:debian local:///opt/spark/examples/jars/spark-examples_2.11-3.0.0-SNAPSHOT.jar 4
18/10/31 11:57:51 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
18/10/31 11:57:51 INFO SparkKubernetesClientFactory: Auto-configuring K8S client using context docker-for-desktop from users K8S config file
18/10/31 11:57:52 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: N/A
	 start time: N/A
	 phase: Pending
	 container status: N/A
18/10/31 11:57:52 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: docker-for-desktop
	 start time: N/A
	 phase: Pending
	 container status: N/A
...
18/10/31 11:58:03 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: docker-for-desktop
	 start time: 2018-10-31T11:57:52Z
	 phase: Succeeded
	 container status:
		 container name: spark-kubernetes-driver
		 container image: rvesse/spark:debian
		 container state: terminated
		 container started at: 2018-10-31T11:57:54Z
		 container finished at: 2018-10-31T11:58:02Z
		 exit code: 0
		 termination reason: Completed
```

Without the `spark.kubernetes.context` setting this will fail because the current context - `minikube` - is pointing to a non-running cluster e.g.

```
> ./spark-submit --master k8s://https://localhost:6443 --deploy-mode cluster --name spark-pi --class org.apache.spark.examples.SparkPi --conf spark.executor.instances=2 --conf spark.kubernetes.container.image=rvesse/spark:debian local:///opt/spark/examples/jars/spark-examples_2.11-3.0.0-SNAPSHOT.jar 4
18/10/31 12:02:30 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
18/10/31 12:02:30 INFO SparkKubernetesClientFactory: Auto-configuring K8S client using current context from users K8S config file
18/10/31 12:02:31 WARN WatchConnectionManager: Exec Failure
javax.net.ssl.SSLHandshakeException: 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.ssl.Alerts.getSSLException(Alerts.java:192)
	at sun.security.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1949)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:302)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:296)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1509)
	at sun.security.ssl.ClientHandshaker.processMessage(ClientHandshaker.java:216)
	at sun.security.ssl.Handshaker.processLoop(Handshaker.java:979)
	at sun.security.ssl.Handshaker.process_record(Handshaker.java:914)
	at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1062)
	at sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1403)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1387)
	at okhttp3.internal.connection.RealConnection.connectTls(RealConnection.java:281)
	at okhttp3.internal.connection.RealConnection.establishProtocol(RealConnection.java:251)
	at okhttp3.internal.connection.RealConnection.connect(RealConnection.java:151)
	at okhttp3.internal.connection.StreamAllocation.findConnection(StreamAllocation.java:195)
	at okhttp3.internal.connection.StreamAllocation.findHealthyConnection(StreamAllocation.java:121)
	at okhttp3.internal.connection.StreamAllocation.newStream(StreamAllocation.java:100)
	at okhttp3.internal.connection.ConnectInterceptor.intercept(ConnectInterceptor.java:42)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.cache.CacheInterceptor.intercept(CacheInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.http.BridgeInterceptor.intercept(BridgeInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RetryAndFollowUpInterceptor.intercept(RetryAndFollowUpInterceptor.java:120)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.BackwardsCompatibilityInterceptor.intercept(BackwardsCompatibilityInterceptor.java:119)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.ImpersonatorInterceptor.intercept(ImpersonatorInterceptor.java:66)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.HttpClientUtils$2.intercept(HttpClientUtils.java:109)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.RealCall.getResponseWithInterceptorChain(RealCall.java:185)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:135)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
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:387)
	at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:292)
	at sun.security.validator.Validator.validate(Validator.java:260)
	at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:324)
	at sun.security.ssl.X509TrustManagerImpl.checkTrusted(X509TrustManagerImpl.java:229)
	at sun.security.ssl.X509TrustManagerImpl.checkServerTrusted(X509TrustManagerImpl.java:124)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1491)
	... 39 more
Caused by: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.provider.certpath.SunCertPathBuilder.build(SunCertPathBuilder.java:141)
	at sun.security.provider.certpath.SunCertPathBuilder.engineBuild(SunCertPathBuilder.java:126)
	at java.security.cert.CertPathBuilder.build(CertPathBuilder.java:280)
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:382)
	... 45 more
Exception in thread "kubernetes-dispatcher-0" Exception in thread "main" java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask611a9c09 rejected from java.util.concurrent.ScheduledThreadPoolExecutor404819e4[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0]
	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2047)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
	at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326)
	at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533)
	at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:632)
	at java.util.concurrent.Executors$DelegatedExecutorService.submit(Executors.java:678)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager.scheduleReconnect(WatchConnectionManager.java:300)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager.access$800(WatchConnectionManager.java:48)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager$2.onFailure(WatchConnectionManager.java:213)
	at okhttp3.internal.ws.RealWebSocket.failWebSocket(RealWebSocket.java:543)
	at okhttp3.internal.ws.RealWebSocket$2.onFailure(RealWebSocket.java:208)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:148)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
io.fabric8.kubernetes.client.KubernetesClientException: Failed to start websocket
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager$2.onFailure(WatchConnectionManager.java:204)
	at okhttp3.internal.ws.RealWebSocket.failWebSocket(RealWebSocket.java:543)
	at okhttp3.internal.ws.RealWebSocket$2.onFailure(RealWebSocket.java:208)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:148)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: javax.net.ssl.SSLHandshakeException: 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.ssl.Alerts.getSSLException(Alerts.java:192)
	at sun.security.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1949)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:302)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:296)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1509)
	at sun.security.ssl.ClientHandshaker.processMessage(ClientHandshaker.java:216)
	at sun.security.ssl.Handshaker.processLoop(Handshaker.java:979)
	at sun.security.ssl.Handshaker.process_record(Handshaker.java:914)
	at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1062)
	at sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1403)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1387)
	at okhttp3.internal.connection.RealConnection.connectTls(RealConnection.java:281)
	at okhttp3.internal.connection.RealConnection.establishProtocol(RealConnection.java:251)
	at okhttp3.internal.connection.RealConnection.connect(RealConnection.java:151)
	at okhttp3.internal.connection.StreamAllocation.findConnection(StreamAllocation.java:195)
	at okhttp3.internal.connection.StreamAllocation.findHealthyConnection(StreamAllocation.java:121)
	at okhttp3.internal.connection.StreamAllocation.newStream(StreamAllocation.java:100)
	at okhttp3.internal.connection.ConnectInterceptor.intercept(ConnectInterceptor.java:42)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.cache.CacheInterceptor.intercept(CacheInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.http.BridgeInterceptor.intercept(BridgeInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RetryAndFollowUpInterceptor.intercept(RetryAndFollowUpInterceptor.java:120)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.BackwardsCompatibilityInterceptor.intercept(BackwardsCompatibilityInterceptor.java:119)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.ImpersonatorInterceptor.intercept(ImpersonatorInterceptor.java:66)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.HttpClientUtils$2.intercept(HttpClientUtils.java:109)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.RealCall.getResponseWithInterceptorChain(RealCall.java:185)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:135)
	... 4 more
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:387)
	at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:292)
	at sun.security.validator.Validator.validate(Validator.java:260)
	at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:324)
	at sun.security.ssl.X509TrustManagerImpl.checkTrusted(X509TrustManagerImpl.java:229)
	at sun.security.ssl.X509TrustManagerImpl.checkServerTrusted(X509TrustManagerImpl.java:124)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1491)
	... 39 more
Caused by: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.provider.certpath.SunCertPathBuilder.build(SunCertPathBuilder.java:141)
	at sun.security.provider.certpath.SunCertPathBuilder.engineBuild(SunCertPathBuilder.java:126)
	at java.security.cert.CertPathBuilder.build(CertPathBuilder.java:280)
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:382)
	... 45 more
18/10/31 12:02:31 INFO ShutdownHookManager: Shutdown hook called
18/10/31 12:02:31 INFO ShutdownHookManager: Deleting directory /private/var/folders/6b/y1010qp107j9w2dhhy8csvz0000xq3/T/spark-5e649891-8a0f-4f17-bf3a-33b34082eba8
```

Suggested reviews: mccheah liyinan926 - this is the follow up fix to the bug discovered while working on SPARK-25809 (PR #22805)

Closes #22904 from rvesse/SPARK-25887.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-22 10:25:21 -08:00
Kazuaki Ishizaki 7bf0794651 [SPARK-26463][CORE] Use ConfigEntry for hardcoded configs for scheduler categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.

## How was this patch tested?

Existing tests

Closes #23416 from kiszk/SPARK-26463.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 07:44:36 -06:00
Jungtaek Lim (HeartSaVioR) 38f030725c [SPARK-26466][CORE] Use ConfigEntry for hardcoded configs for submit categories.
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use `ConfigEntry`.

* spark.kryo
* spark.kryoserializer
* spark.serializer
* spark.jars
* spark.files
* spark.submit
* spark.deploy
* spark.worker

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).

## How was this patch tested?

Existing tests.

Closes #23532 from HeartSaVioR/SPARK-26466-v2.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-16 20:57:21 -06:00
Jungtaek Lim (HeartSaVioR) d9e4cf67c0 [SPARK-26482][CORE] Use ConfigEntry for hardcoded configs for ui categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use `ConfigEntry`.

* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).

## How was this patch tested?

Existing tests.

Closes #23423 from HeartSaVioR/SPARK-26466.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-11 10:18:07 -08:00
Marcelo Vanzin 669e8a1559 [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens.
This change modifies the behavior of the delegation token code when running
on YARN, so that the driver controls the renewal, in both client and cluster
mode. For that, a few different things were changed:

* The AM code only runs code that needs DTs when DTs are available.

In a way, this restores the AM behavior to what it was pre-SPARK-23361, but
keeping the fix added in that bug. Basically, all the AM code is run in a
"UGI.doAs()" block; but code that needs to talk to HDFS (basically the
distributed cache handling code) was delayed to the point where the driver
is up and running, and thus when valid delegation tokens are available.

* SparkSubmit / ApplicationMaster now handle user login, not the token manager.

The previous AM code was relying on the token manager to keep the user
logged in when keytabs are used. This required some odd APIs in the token
manager and the AM so that the right UGI was exposed and used in the right
places.

After this change, the logged in user is handled separately from the token
manager, so the API was cleaned up, and, as explained above, the whole AM
runs under the logged in user, which also helps with simplifying some more code.

* Distributed cache configs are sent separately to the AM.

Because of the delayed initialization of the cached resources in the AM, it
became easier to write the cache config to a separate properties file instead
of bundling it with the rest of the Spark config. This also avoids having
to modify the SparkConf to hide things from the UI.

* Finally, the AM doesn't manage the token manager anymore.

The above changes allow the token manager to be completely handled by the
driver's scheduler backend code also in YARN mode (whether client or cluster),
making it similar to other RMs. To maintain the fix added in SPARK-23361 also
in client mode, the AM now sends an extra message to the driver on initialization
to fetch delegation tokens; and although it might not really be needed, the
driver also keeps the running AM updated when new tokens are created.

Tested in a kerberized cluster with the same tests used to validate SPARK-23361,
in both client and cluster mode. Also tested with a non-kerberized cluster.

Closes #23338 from vanzin/SPARK-25689.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-01-07 14:40:08 -06:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
Takuya UESHIN 4419e1daca [SPARK-26445][CORE] Use ConfigEntry for hardcoded configs for driver/executor categories.
## What changes were proposed in this pull request?

The PR makes hardcoded spark.driver, spark.executor, and spark.cores.max configs to use `ConfigEntry`.

Note that some config keys are from `SparkLauncher` instead of defining in the config package object because the string is already defined in it and it does not depend on core module.

## How was this patch tested?

Existing tests.

Closes #23415 from ueshin/issues/SPARK-26445/hardcoded_driver_executor_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-04 22:12:35 +08:00
Jungtaek Lim (HeartSaVioR) 05372d188a [SPARK-26489][CORE] Use ConfigEntry for hardcoded configs for python/r categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use ConfigEntry.

* spark.pyspark
* spark.python
* spark.r

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties, python source code)

## How was this patch tested?

Existing tests.

Closes #23428 from HeartSaVioR/SPARK-26489.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-03 14:30:27 -08:00
Marcelo Vanzin 4b3fe3a9cc [SPARK-25815][K8S] Support kerberos in client mode, keytab-based token renewal.
This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode, and keytab-based token
renewal is enabled.

The change re-works the k8s feature steps related to kerberos so
that the driver does all the credential management and provides all
the needed information to executors - so nothing needs to be added
to executor pods. This also makes cluster mode behave a lot more
similarly to client mode, since no driver-related config steps are run
in the latter case.

The main two things that don't need to happen in executors anymore are:

- adding the Hadoop config to the executor pods: this is not needed
  since the Spark driver will serialize the Hadoop config and send
  it to executors when running tasks.

- mounting the kerberos config file in the executor pods: this is
  not needed once you remove the above. The Hadoop conf sent by
  the driver with the tasks is already resolved (i.e. has all the
  kerberos names properly defined), so executors do not need access
  to the kerberos realm information anymore.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:
- client and cluster mode with kinit
- cluster mode with keytab
- cluster mode with local: keytab
- YARN cluster with keytab (to make sure it isn't broken)

Closes #22911 from vanzin/SPARK-25815.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-18 13:30:09 -08:00
suxingfate 114d0de14c [SPARK-25922][K8] Spark Driver/Executor "spark-app-selector" label mismatch
## What changes were proposed in this pull request?

In K8S Cluster mode, the algorithm to generate spark-app-selector/spark.app.id of spark driver is different with spark executor.
This patch makes sure spark driver and executor to use the same spark-app-selector/spark.app.id if spark.app.id is set, otherwise it will use superclass applicationId.

In K8S Client mode, spark-app-selector/spark.app.id for executors will use superclass applicationId.

## How was this patch tested?

Manually run."

Closes #23322 from suxingfate/SPARK-25922.

Lead-authored-by: suxingfate <suxingfate@163.com>
Co-authored-by: xinglwang <xinglwang@ebay.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-12-17 13:36:57 -08:00
Marcelo Vanzin a63e7b2a21 [SPARK-25877][K8S] Move all feature logic to feature classes.
This change makes the driver and executor builders a lot simpler
by encapsulating almost all feature logic into the respective
feature classes. The only logic that remains is the creation of
the initial pod, which needs to happen before anything else so
is better to be left in the builder class.

Most feature classes already behave fine when the config has nothing
they should handle, but a few minor tweaks had to be added. Unit
tests were also updated or added to account for these.

The builder suites were simplified a lot and just test the remaining
pod-related code in the builders themselves.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23220 from vanzin/SPARK-25877.
2018-12-12 12:01:21 -08:00
mcheah 57d6fbfa8c [SPARK-26239] File-based secret key loading for SASL.
This proposes an alternative way to load secret keys into a Spark application that is running on Kubernetes. Instead of automatically generating the secret, the secret key can reside in a file that is shared between both the driver and executor containers.

Unit tests.

Closes #23252 from mccheah/auth-secret-with-file.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-11 13:50:16 -08:00
Marcelo Vanzin dbd90e5440 [SPARK-26194][K8S] Auto generate auth secret for k8s apps.
This change modifies the logic in the SecurityManager to do two
things:

- generate unique app secrets also when k8s is being used
- only store the secret in the user's UGI on YARN

The latter is needed so that k8s won't unnecessarily create
k8s secrets for the UGI credentials when only the auth token
is stored there.

On the k8s side, the secret is propagated to executors using
an environment variable instead. This ensures it works in both
client and cluster mode.

Security doc was updated to mention the feature and clarify that
proper access control in k8s should be enabled for it to be secure.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23174 from vanzin/SPARK-26194.
2018-12-06 14:17:13 -08:00
Stavros Kontopoulos a24e1a126c [SPARK-26256][K8S] Fix labels for pod deletion
## What changes were proposed in this pull request?
Adds proper labels when deleting executor pods.

## How was this patch tested?
Manually with tests.

Closes #23209 from skonto/fix-deletion-labels.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-03 14:57:18 -08:00
Stavros Kontopoulos 0c2935b01d [SPARK-25515][K8S] Adds a config option to keep executor pods for debugging
## What changes were proposed in this pull request?
Keeps K8s executor resources present if case of failure or normal termination.
Introduces a new boolean config option: `spark.kubernetes.deleteExecutors`, with default value set to true.
The idea is to update Spark K8s backend structures but leave the resources around.
The assumption is that since entries are not removed from the `removedExecutorsCache` we are immune to updates that refer to the the executor resources previously removed.
The only delete operation not touched is the one in the `doKillExecutors` method.
Reason is right now we dont support [blacklisting](https://issues.apache.org/jira/browse/SPARK-23485) and dynamic allocation with Spark on K8s. In both cases in the future we might want to handle these scenarios although its more complicated.
More tests can be added if approach is approved.
## How was this patch tested?
Manually by running a Spark job and verifying pods are not deleted.

Closes #23136 from skonto/keep_pods.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-12-03 09:02:47 -08:00
Marcelo Vanzin 6be272b75b [SPARK-25876][K8S] Simplify kubernetes configuration types.
There are a few issues with the current configuration types used in
the kubernetes backend:

- they use type parameters for role-specific specialization, which makes
  type signatures really noisy throughout the code base.

- they break encapsulation by forcing the code that creates the config
  object to remove the configuration from SparkConf before creating the
  k8s-specific wrapper.

- they don't provide an easy way for tests to have default values for
  fields they do not use.

This change fixes those problems by:

- creating a base config type with role-specific specialization using
  inheritance

- encapsulating the logic of parsing SparkConf into k8s-specific views
  inside the k8s config classes

- providing some helper code for tests to easily override just the part
  of the configs they want.

Most of the change relates to the above, especially cleaning up the
tests. While doing that, I also made some smaller changes elsewhere:

- removed unnecessary type parameters in KubernetesVolumeSpec

- simplified the error detection logic in KubernetesVolumeUtils; all
  the call sites would just throw the first exception collected by
  that class, since they all called "get" on the "Try" object. Now
  the unnecessary wrapping is gone and the exception is just thrown
  where it occurs.

- removed a lot of unnecessary mocking from tests.

- changed the kerberos-related code so that less logic needs to live
  in the driver builder. In spirit it should be part of the upcoming
  work in this series of cleanups, but it made parts of this change
  simpler.

Tested with existing unit tests and integration tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #22959 from vanzin/SPARK-25876.
2018-11-30 16:23:37 -08:00
Nihar Sheth 3df307aa51 [SPARK-25960][K8S] Support subpath mounting with Kubernetes
## What changes were proposed in this pull request?

This PR adds configurations to use subpaths with Spark on k8s. Subpaths (https://kubernetes.io/docs/concepts/storage/volumes/#using-subpath) allow the user to specify a path within a volume to use instead of the volume's root.

## How was this patch tested?

Added unit tests. Ran SparkPi on a cluster with event logging pointed at a subpath-mount and verified the driver host created and used the subpath.

Closes #23026 from NiharS/k8s_subpath.

Authored-by: Nihar Sheth <niharrsheth@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 11:06:02 -08:00
Sean Owen 32365f8177 [SPARK-26090][CORE][SQL][ML] Resolve most miscellaneous deprecation and build warnings for Spark 3
## What changes were proposed in this pull request?

The build has a lot of deprecation warnings. Some are new in Scala 2.12 and Java 11. We've fixed some, but I wanted to take a pass at fixing lots of easy miscellaneous ones here.

They're too numerous and small to list here; see the pull request. Some highlights:

- `BeanInfo` is deprecated in 2.12, and BeanInfo classes are pretty ancient in Java. Instead, case classes can explicitly declare getters
- Eta expansion of zero-arg methods; foo() becomes () => foo() in many cases
- Floating-point Range is inexact and deprecated, like 0.0 to 100.0 by 1.0
- finalize() is finally deprecated (just needs to be suppressed)
- StageInfo.attempId was deprecated and easiest to remove here

I'm not now going to touch some chunks of deprecation warnings:

- Parquet deprecations
- Hive deprecations (particularly serde2 classes)
- Deprecations in generated code (mostly Thriftserver CLI)
- ProcessingTime deprecations (we may need to revive this class as internal)
- many MLlib deprecations because they concern methods that may be removed anyway
- a few Kinesis deprecations I couldn't figure out
- Mesos get/setRole, which I don't know well
- Kafka/ZK deprecations (e.g. poll())
- Kinesis
- a few other ones that will probably resolve by deleting a deprecated method

## How was this patch tested?

Existing tests, including manual testing with the 2.11 build and Java 11.

Closes #23065 from srowen/SPARK-26090.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 09:16:42 -06:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Marcelo Vanzin 3404a73f4c [SPARK-25875][K8S] Merge code to set up driver command into a single step.
Right now there are 3 different classes dealing with building the driver
command to run inside the pod, one for each "binding" supported by Spark.
This has two main shortcomings:

- the code in the 3 classes is very similar; changing things in one place
  would probably mean making a similar change in the others.

- it gives the false impression that the step implementation is the only
  place where binding-specific logic is needed. That is not true; there
  was code in KubernetesConf that was binding-specific, and there's also
  code in the executor-specific config step. So the 3 classes weren't really
  working as a language-specific abstraction.

On top of that, the current code was propagating command line parameters in
a different way depending on the binding. That doesn't seem necessary, and
in fact using environment variables for command line parameters is in general
a really bad idea, since you can't handle special characters (e.g. spaces)
that way.

This change merges the 3 different code paths for Java, Python and R into
a single step, and also merges the 3 code paths to start the Spark driver
in the k8s entry point script. This increases the amount of shared code,
and also moves more feature logic into the step itself, so it doesn't live
in KubernetesConf.

Note that not all logic related to setting up the driver lives in that
step. For example, the memory overhead calculation still lives separately,
except it now happens in the driver config step instead of outside the
step hierarchy altogether.

Some of the noise in the diff is because of changes to KubernetesConf, which
will be addressed in a separate change.

Tested with new and updated unit tests + integration tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #22897 from vanzin/SPARK-25875.
2018-11-02 13:58:08 -07:00
Rob Vesse fc8222298e [SPARK-25809][K8S][TEST] New K8S integration testing backends
## What changes were proposed in this pull request?

Currently K8S integration tests are hardcoded to use a `minikube` based backend.  `minikube` is VM based so can be resource hungry and also doesn't cope well with certain networking setups (for example using Cisco AnyConnect software VPN `minikube` is unusable as it detects its own IP incorrectly).

This PR Adds a new K8S integration testing backend that allows for using the Kubernetes support in [Docker for Desktop](https://blog.docker.com/2018/07/kubernetes-is-now-available-in-docker-desktop-stable-channel/).  It also generalises the framework to be able to run the integration tests against an arbitrary Kubernetes cluster.

To Do:

- [x] General Kubernetes cluster backend
- [x] Documentation on Kubernetes integration testing
- [x] Testing of general K8S backend
- [x] Check whether change from timestamps being `Time` to `String` in Fabric 8 upgrade needs additional fix up

## How was this patch tested?

Ran integration tests with Docker for Desktop and all passed:

![screen shot 2018-10-23 at 14 19 56](https://user-images.githubusercontent.com/2104864/47363460-c5816a00-d6ce-11e8-9c15-56b34698e797.png)

Suggested Reviewers: ifilonenko srowen

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22805 from rvesse/SPARK-25809.
2018-11-01 09:33:55 -07:00
Marcelo Vanzin 68dde3481e [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager.
This avoids having two classes to deal with tokens; now the above
class is a one-stop shop for dealing with delegation tokens. The
YARN backend extends that class instead of doing composition like
before, resulting in a bit less code there too.

The renewer functionality is basically the same code that used to
be in YARN's AMCredentialRenewer. That is also the reason why the
public API of HadoopDelegationTokenManager is a little bit odd;
the YARN AM has some odd requirements for how this all should be
initialized, and the weirdness is needed currently to support that.

Tested:
- YARN with stress app for DT renewal
- Mesos and K8S with basic kerberos tests (both tgt and keytab)

Closes #22624 from vanzin/SPARK-23781.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-31 13:00:10 -05:00
Onur Satici f6cc354d83 [SPARK-24434][K8S] pod template files
## What changes were proposed in this pull request?

New feature to pass podspec files for driver and executor pods.

## How was this patch tested?
new unit and integration tests

- [x] more overwrites in integration tests
- [ ] invalid template integration test, documentation

Author: Onur Satici <osatici@palantir.com>
Author: Yifei Huang <yifeih@palantir.com>
Author: onursatici <onursatici@gmail.com>

Closes #22146 from onursatici/pod-template.
2018-10-30 13:52:44 -07:00
Ilan Filonenko e9b71c8f01 [SPARK-25828][K8S] Bumping Kubernetes-Client version to 4.1.0
## What changes were proposed in this pull request?

Changed the `kubernetes-client` version and refactored code that broke as a result

## How was this patch tested?

Unit and Integration tests

Closes #22820 from ifilonenko/SPARK-25828.

Authored-by: Ilan Filonenko <ifilondz@gmail.com>
Signed-off-by: Erik Erlandson <eerlands@redhat.com>
2018-10-26 15:59:12 -07:00
Ilan Filonenko 19ada15d1b [SPARK-24516][K8S] Change Python default to Python3
## What changes were proposed in this pull request?

As this is targeted for 3.0.0 and Python2 will be deprecated by Jan 1st, 2020, I feel it is appropriate to change the default to Python3. Especially as these projects [found here](https://python3statement.org/) are deprecating their support.

## How was this patch tested?

Unit and Integration tests

Author: Ilan Filonenko <ifilondz@gmail.com>

Closes #22810 from ifilonenko/SPARK-24516.
2018-10-24 23:29:47 -07:00
Mike Kaplinskiy ffe256ce16 [SPARK-25730][K8S] Delete executor pods from kubernetes after figuring out why they died
## What changes were proposed in this pull request?

`removeExecutorFromSpark` tries to fetch the reason the executor exited from Kubernetes, which may be useful if the pod was OOMKilled. However, the code previously deleted the pod from Kubernetes first which made retrieving this status impossible. This fixes the ordering.

On a separate but related note, it would be nice to wait some time before removing the pod - to let the operator examine logs and such.

## How was this patch tested?

Running on my local cluster.

Author: Mike Kaplinskiy <mike.kaplinskiy@gmail.com>

Closes #22720 from mikekap/patch-1.
2018-10-21 11:32:33 -07:00
Ilan Filonenko 6c9c84ffb9 [SPARK-23257][K8S] Kerberos Support for Spark on K8S
## What changes were proposed in this pull request?
This is the work on setting up Secure HDFS interaction with Spark-on-K8S.
The architecture is discussed in this community-wide google [doc](https://docs.google.com/document/d/1RBnXD9jMDjGonOdKJ2bA1lN4AAV_1RwpU_ewFuCNWKg)
This initiative can be broken down into 4 Stages

**STAGE 1**
- [x] Detecting `HADOOP_CONF_DIR` environmental variable and using Config Maps to store all Hadoop config files locally, while also setting `HADOOP_CONF_DIR` locally in the driver / executors

**STAGE 2**
- [x] Grabbing `TGT` from `LTC` or using keytabs+principle and creating a `DT` that will be mounted as a secret or using a pre-populated secret

**STAGE 3**
- [x] Driver

**STAGE 4**
- [x] Executor

## How was this patch tested?
Locally tested on a single-noded, pseudo-distributed Kerberized Hadoop Cluster
- [x] E2E Integration tests https://github.com/apache/spark/pull/22608
- [ ] Unit tests

## Docs and Error Handling?
- [x] Docs
- [x] Error Handling

## Contribution Credit
kimoonkim skonto

Closes #21669 from ifilonenko/secure-hdfs.

Lead-authored-by: Ilan Filonenko <if56@cornell.edu>
Co-authored-by: Ilan Filonenko <ifilondz@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 15:48:51 -07:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
Prashant Sharma 4da541a5d2
[SPARK-25543][K8S] Print debug message iff execIdsRemovedInThisRound is not empty.
## What changes were proposed in this pull request?

Spurious logs like /sec.
2018-09-26 09:33:57 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids  from Spark that were either found to be deleted or non-existent in the cluster.
2018-09-26 09:33:58 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids  from Spark that were either found to be deleted or non-existent in the cluster.
2018-09-26 09:33:59 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids  from Spark that were either found to be deleted or non-existent in the cluster.
2018-09-26 09:34:00 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids  from Spark that were either found to be deleted or non-existent in the cluster.

The fix is easy, first check if there are any removed executors, before producing the log message.

## How was this patch tested?

Tested by manually deploying to a minikube cluster.

Closes #22565 from ScrapCodes/spark-25543/k8s/debug-log-spurious-warning.

Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-30 14:28:20 -07:00
hyukjinkwon a2f502cf53 [SPARK-25565][BUILD] Add scalastyle rule to check add Locale.ROOT to .toLowerCase and .toUpperCase for internal calls
## What changes were proposed in this pull request?

This PR adds a rule to force `.toLowerCase(Locale.ROOT)` or `toUpperCase(Locale.ROOT)`.

It produces an error as below:

```
[error]       Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you
[error]       should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead.
[error]       If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with
[error]       // scalastyle:off caselocale
[error]       .toUpperCase
[error]       .toLowerCase
[error]       // scalastyle:on caselocale
```

This PR excludes the cases above for SQL code path for external calls like table name, column name and etc.

For test suites, or when it's clear there's no locale problem like Turkish locale problem, it uses `Locale.ROOT`.

One minor problem is, `UTF8String` has both methods, `toLowerCase` and `toUpperCase`, and the new rule detects them as well. They are ignored.

## How was this patch tested?

Manually tested, and Jenkins tests.

Closes #22581 from HyukjinKwon/SPARK-25565.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-30 14:31:04 +08:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
Stavros Kontopoulos 3e75a9fa24 [SPARK-25295][K8S] Fix executor names collision
## What changes were proposed in this pull request?
Fixes the collision issue with spark executor names in client mode, see SPARK-25295 for the details.
It follows the cluster name convention as app-name will be used as the prefix and if that is not defined we use "spark" as the default prefix. Eg. `spark-pi-1536781360723-exec-1` where spark-pi is the name of the app passed at the config side or transformed if it contains illegal characters.

Also fixes the issue with spark app name having spaces in cluster mode.
If you run the Spark Pi test in client mode it passes.
The tricky part is the user may set the app name:
3030b82c89/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala (L30)
If i do:

```
./bin/spark-submit
...
 --deploy-mode cluster --name "spark pi"
...
```
it will fail as the app name is used for the prefix of driver's pod name and it cannot have spaces (according to k8s conventions).

## How was this patch tested?

Manually by running spark job in client mode.
To reproduce do:
```
kubectl create -f service.yaml
kubectl create -f pod.yaml
```
 service.yaml :
```
kind: Service
apiVersion: v1
metadata:
  name: spark-test-app-1-svc
spec:
  clusterIP: None
  selector:
    spark-app-selector: spark-test-app-1
  ports:
  - protocol: TCP
    name: driver-port
    port: 7077
    targetPort: 7077
  - protocol: TCP
    name: block-manager
    port: 10000
    targetPort: 10000
```
pod.yaml:

```
apiVersion: v1
kind: Pod
metadata:
  name: spark-test-app-1
  labels:
    spark-app-selector: spark-test-app-1
spec:
  containers:
  - name: spark-test
    image: skonto/spark:k8s-client-fix
    imagePullPolicy: Always
    command:
      - 'sh'
      - '-c'
      -  "/opt/spark/bin/spark-submit
              --verbose
              --master k8s://https://kubernetes.default.svc
              --deploy-mode client
              --class org.apache.spark.examples.SparkPi
              --conf spark.app.name=spark
              --conf spark.executor.instances=1
              --conf spark.kubernetes.container.image=skonto/spark:k8s-client-fix
              --conf spark.kubernetes.container.image.pullPolicy=Always
              --conf spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/kubernetes.io/serviceaccount/token
              --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/kubernetes.io/serviceaccount/ca.crt
              --conf spark.executor.memory=500m
              --conf spark.executor.cores=1
              --conf spark.executor.instances=1
              --conf spark.driver.host=spark-test-app-1-svc.default.svc
              --conf spark.driver.port=7077
              --conf spark.driver.blockManager.port=10000
              local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar 1000000"
```

Closes #22405 from skonto/fix-k8s-client-mode-executor-names.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-09-12 22:02:59 -07:00
Ilan Filonenko 1cfda44825 [SPARK-25021][K8S] Add spark.executor.pyspark.memory limit for K8S
## What changes were proposed in this pull request?

Add spark.executor.pyspark.memory limit for K8S

## How was this patch tested?

Unit and Integration tests

Closes #22298 from ifilonenko/SPARK-25021.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2018-09-08 22:18:06 -07:00
Rob Vesse da6fa3828b [SPARK-25262][K8S] Allow SPARK_LOCAL_DIRS to be tmpfs backed on K8S
## What changes were proposed in this pull request?

The default behaviour of Spark on K8S currently is to create `emptyDir` volumes to back `SPARK_LOCAL_DIRS`.  In some environments e.g. diskless compute nodes this may actually hurt performance because these are backed by the Kubelet's node storage which on a diskless node will typically be some remote network storage.

Even if this is enterprise grade storage connected via a high speed interconnect the way Spark uses these directories as scratch space (lots of relatively small short lived files) has been observed to cause serious performance degradation.  Therefore we would like to provide the option to use K8S's ability to instead back these `emptyDir` volumes with `tmpfs`. Therefore this PR adds a configuration option that enables `SPARK_LOCAL_DIRS` to be backed by Memory backed `emptyDir` volumes rather than the default.

Documentation is added to describe both the default behaviour plus this new option and its implications.  One of which is that scratch space then counts towards your pods memory limits and therefore users will need to adjust their memory requests accordingly.

*NB* - This is an alternative version of PR #22256 reduced to just the `tmpfs` piece

## How was this patch tested?

Ran with this option in our diskless compute environments to verify functionality

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22323 from rvesse/SPARK-25262-tmpfs.
2018-09-06 16:18:59 -07:00
Rob Vesse 27d3b0a51c [SPARK-25222][K8S] Improve container status logging
## What changes were proposed in this pull request?

Currently when running Spark on Kubernetes a logger is run by the client that watches the K8S API for events related to the Driver pod and logs them.  However for the container status aspect of the logging this simply dumps the raw object which is not human readable e.g.

![screen shot 2018-08-24 at 10 37 46](https://user-images.githubusercontent.com/2104864/44577799-e0486880-a789-11e8-9ae9-fdeddacbbea8.png)
![screen shot 2018-08-24 at 10 38 14](https://user-images.githubusercontent.com/2104864/44577800-e0e0ff00-a789-11e8-81f5-3bb315dbbdb1.png)

This is despite the fact that the logging class in question actually has methods to pretty print this information but only invokes these at the end of a job.

This PR improves the logging to always use the pretty printing methods, additionally modifying them to include further useful information provided by the K8S API.

A similar issue also exists when tasks are lost that will be addressed by further commits to this PR

- [x] Improved `LoggingPodStatusWatcher`
- [x] Improved container status on task failure

## How was this patch tested?

Built and launched jobs with the updated Spark client and observed the new human readable output:

![screen shot 2018-08-24 at 11 09 32](https://user-images.githubusercontent.com/2104864/44579429-5353de00-a78e-11e8-9228-c750af8e6311.png)
![screen shot 2018-08-24 at 11 09 42](https://user-images.githubusercontent.com/2104864/44579430-5353de00-a78e-11e8-8fce-d5bb2a3ae65f.png)
![screen shot 2018-08-24 at 11 10 13](https://user-images.githubusercontent.com/2104864/44579431-53ec7480-a78e-11e8-9fa2-aeabc5b28ec4.png)
![screen shot 2018-08-24 at 17 47 44](https://user-images.githubusercontent.com/2104864/44596922-db090f00-a7c5-11e8-910c-bc2339f5a196.png)

Suggested reviewers: liyinan926 mccheah

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22215 from rvesse/SPARK-25222.
2018-09-06 16:15:11 -07:00
Ilan Filonenko e1d72f2c07 [SPARK-25264][K8S] Fix comma-delineated arguments passed into PythonRunner and RRunner
## What changes were proposed in this pull request?

Fixes the issue brought up in https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/issues/273 where the arguments were being comma-delineated, which was incorrect wrt to the PythonRunner and RRunner.

## How was this patch tested?

Modified unit test to test this change.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22257 from ifilonenko/SPARK-25264.
2018-08-31 15:46:45 -07:00
Ilan Filonenko ba84bcb2c4 [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s
## What changes were proposed in this pull request?

Introducing R Bindings for Spark R on K8s

- [x] Running SparkR Job

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration Tests

## Example:

Commands to run example spark job:
1. `dev/make-distribution.sh --pip --r --tgz -Psparkr -Phadoop-2.7 -Pkubernetes`
2. `bin/docker-image-tool.sh -m -t testing build`
3.
```
bin/spark-submit \
    --master k8s://https://192.168.64.33:8443 \
    --deploy-mode cluster \
    --name spark-r \
    --conf spark.executor.instances=1 \
    --conf spark.kubernetes.container.image=spark-r:testing \
    local:///opt/spark/examples/src/main/r/dataframe.R
```

This above spark-submit command works given the distribution. (Will include this integration test in PR once PRB is ready).

Author: Ilan Filonenko <if56@cornell.edu>

Closes #21584 from ifilonenko/spark-r.
2018-08-17 16:04:02 -07:00
Ilan Filonenko a791c29bd8 [SPARK-23984][K8S] Changed Python Version config to be camelCase
## What changes were proposed in this pull request?

Small formatting change to have Python Version be camelCase as per request during PR review.

## How was this patch tested?

Tested with unit and integration tests

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22095 from ifilonenko/spark-py-edits.
2018-08-15 17:52:12 -07:00
Adelbert Chang f5113ea8d7 [SPARK-24960][K8S] explicitly expose ports on driver container
https://issues.apache.org/jira/browse/SPARK-24960
## What changes were proposed in this pull request?

Expose ports explicitly in the driver container. The driver Service created expects to reach the driver Pod at specific ports which before this change, were not explicitly exposed and would likely cause connection issues (see https://github.com/apache-spark-on-k8s/spark/issues/617).

This is a port of the original PR created in the now-deprecated Kubernetes fork: https://github.com/apache-spark-on-k8s/spark/pull/618

## How was this patch tested?

Failure in https://github.com/apache-spark-on-k8s/spark/issues/617 reproduced on Kubernetes 1.6.x and 1.8.x. Built the driver image with this patch and observed fixed https://github.com/apache-spark-on-k8s/spark/issues/617 on Kubernetes 1.6.x.

Author: Adelbert Chang <Adelbert.Chang@target.com>

Closes #21884 from adelbertc/k8s-expose-driver-ports.
2018-08-01 13:57:33 -07:00
mcheah 571a6f0574 [SPARK-23146][K8S] Support client mode.
## What changes were proposed in this pull request?

Support client mode for the Kubernetes scheduler.

Client mode works more or less identically to cluster mode. However, in client mode, the Spark Context needs to be manually bootstrapped with certain properties which would have otherwise been set up by spark-submit in cluster mode. Specifically:

- If the user doesn't provide a driver pod name, we don't add an owner reference. This is for usage when the driver is not running in a pod in the cluster. In such a case, the driver can only provide a best effort to clean up the executors when the driver exits, but cleaning up the resources is not guaranteed. The executor JVMs should exit if the driver JVM exits, but the pods will still remain in the cluster in a COMPLETED or FAILED state.
- The user must provide a host (spark.driver.host) and port (spark.driver.port) that the executors can connect to. When using spark-submit in cluster mode, spark-submit generates the headless service automatically; in client mode, the user is responsible for setting up their own connectivity.

We also change the authentication configuration prefixes for client mode.

## How was this patch tested?

Adding an integration test to exercise client mode support.

Author: mcheah <mcheah@palantir.com>

Closes #21748 from mccheah/k8s-client-mode.
2018-07-25 11:08:41 -07:00
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

This PR enables a Java bytecode check tool [spotbugs](https://spotbugs.github.io/) to avoid possible integer overflow at multiplication. When an violation is detected, the build process is stopped.
Due to the tool limitation, some other checks will be enabled. In this PR, [these patterns](http://spotbugs-in-kengo-toda.readthedocs.io/en/lqc-list-detectors/detectors.html#findpuzzlers) in `FindPuzzlers` can be detected.

This check is enabled at `compile` phase. Thus, `mvn compile` or `mvn package` launches this check.

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21542 from kiszk/SPARK-24529.
2018-07-12 09:52:23 +08:00
Andrew Korzhuev 5ff1b9ba19 [SPARK-23529][K8S] Support mounting volumes
This PR continues #21095 and intersects with #21238. I've added volume mounts as a separate step and added PersistantVolumeClaim support.

There is a fundamental problem with how we pass the options through spark conf to fabric8. For each volume type and all possible volume options we would have to implement some custom code to map config values to fabric8 calls. This will result in big body of code we would have to support and means that Spark will always be somehow out of sync with k8s.

I think there needs to be a discussion on how to proceed correctly (eg use PodPreset instead)

----

Due to the complications of provisioning and managing actual resources this PR addresses only volume mounting of already present resources.

----
- [x] emptyDir support
- [x] Testing
- [x] Documentation
- [x] KubernetesVolumeUtils tests

Author: Andrew Korzhuev <andrew.korzhuev@klarna.com>
Author: madanadit <adit@alluxio.com>

Closes #21260 from andrusha/k8s-vol.
2018-07-10 22:53:44 -07:00
Stavros Kontopoulos 85fe1297e3 [SPARK-24428][K8S] Fix unused code
## What changes were proposed in this pull request?

Remove code that is misleading and is a leftover from a previous implementation.

## How was this patch tested?
Manually.

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #21462 from skonto/fix-k8s-docs.
2018-07-02 13:08:16 -07:00