From 6ddb0579d5f45d7984df991f4f64c3ce15af31d2 Mon Sep 17 00:00:00 2001 From: IceMimosa Date: Sat, 23 Mar 2019 12:07:13 +0800 Subject: [PATCH] Merge master code, see https://growingio.atlassian.net/wiki/spaces/DATA/pages/864879077/on+K8S MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix ImplicitCastInputTypes [SPARK-25222][K8S] Improve container status logging [SPARK-25262][K8S] Allow SPARK_LOCAL_DIRS to be tmpfs backed on K8S [SPARK-25021][K8S] Add spark.executor.pyspark.memory limit for K8S [SPARK-25415][SQL] Make plan change log in RuleExecutor configurable by SQLConf In RuleExecutor, after applying a rule, if the plan has changed, the before and after plan will be logged using level "trace". At times, however, such information can be very helpful for debugging. Hence, making the log level configurable in SQLConf would allow users to turn on the plan change log independently and save the trouble of tweaking log4j settings. Meanwhile, filtering plan change log for specific rules can also be very useful. So this PR adds two SQL configurations: 1. spark.sql.optimizer.planChangeLog.level - set a specific log level for logging plan changes after a rule is applied. 2. spark.sql.optimizer.planChangeLog.rules - enable plan change logging only for a set of specified rules, separated by commas. Added UT. Closes #22406 from maryannxue/spark-25415. Authored-by: maryannxue Signed-off-by: gatorsmile [SPARK-25338][TEST] Ensure to call super.beforeAll() and super.afterAll() in test cases This PR ensures to call `super.afterAll()` in `override afterAll()` method for test suites. * Some suites did not call `super.afterAll()` * Some suites may call `super.afterAll()` only under certain condition * Others never call `super.afterAll()`. This PR also ensures to call `super.beforeAll()` in `override beforeAll()` for test suites. Existing UTs Closes #22337 from kiszk/SPARK-25338. Authored-by: Kazuaki Ishizaki Signed-off-by: Dongjoon Hyun [SPARK-25415][SQL][FOLLOW-UP] Add Locale.ROOT when toUpperCase Add `Locale.ROOT` when `toUpperCase`. manual tests Closes #22531 from wangyum/SPARK-25415. Authored-by: Yuming Wang Signed-off-by: hyukjinkwon [SPARK-25514][SQL] Generating pretty JSON by to_json The PR introduces new JSON option `pretty` which allows to turn on `DefaultPrettyPrinter` of `Jackson`'s Json generator. New option is useful in exploring of deep nested columns and in converting of JSON columns in more readable representation (look at the added test). Added rount trip test which convert an JSON string to pretty representation via `from_json()` and `to_json()`. Closes #22534 from MaxGekk/pretty-json. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: hyukjinkwon [SPARK-25262][DOC][FOLLOWUP] Fix missing markup tag This adds a missing end markup tag. This should go `master` branch only. This is a doc-only change. Manual via `SKIP_API=1 jekyll build`. Closes #22584 from dongjoon-hyun/SPARK-25262. Authored-by: Dongjoon Hyun Signed-off-by: hyukjinkwon [SPARK-23257][K8S] Kerberos Support for Spark on K8S [SPARK-25682][K8S] Package example jars in same target for dev and distro images. This way the image generated from both environments has the same layout, with just a difference in contents that should not affect functionality. Also added some minor error checking to the image script. Closes #22681 from vanzin/SPARK-25682. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin [SPARK-25745][K8S] Improve docker-image-tool.sh script Adds error checking and handling to `docker` invocations ensuring the script terminates early in the event of any errors. This avoids subtle errors that can occur e.g. if the base image fails to build the Python/R images can end up being built from outdated base images and makes it more explicit to the user that something went wrong. Additionally the provided `Dockerfiles` assume that Spark was first built locally or is a runnable distribution however it didn't previously enforce this. The script will now check the JARs folder to ensure that Spark JARs actually exist and if not aborts early reminding the user they need to build locally first. - Tested with a `mvn clean` working copy and verified that the script now terminates early - Tested with bad `Dockerfiles` that fail to build to see that early termination occurred Closes #22748 from rvesse/SPARK-25745. Authored-by: Rob Vesse Signed-off-by: Marcelo Vanzin [SPARK-25730][K8S] Delete executor pods from kubernetes after figuring out why they died `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. Running on my local cluster. Author: Mike Kaplinskiy Closes #22720 from mikekap/patch-1. [SPARK-25828][K8S] Bumping Kubernetes-Client version to 4.1. [SPARK-24434][K8S] pod template files [SPARK-25809][K8S][TEST] New K8S integration testing backends [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 Closes #22897 from vanzin/SPARK-25875. [SPARK-25897][K8S] Hook up k8s integration tests to sbt build. The integration tests can now be run in sbt if the right profile is enabled, using the "test" task under the respective project. This avoids having to fall back to maven to run the tests, which invalidates all your compiled stuff when you go back to sbt, making development way slower than it should. There's also a task to run the tests directly without refreshing the docker images, which is helpful if you just made a change to the submission code which should not affect the code in the images. The sbt tasks currently are not very customizable; there's some very minor things you can set in the sbt shell itself, but otherwise it's hardcoded to run on minikube. I also had to make some slight adjustments to the IT code itself, mostly to remove assumptions about the existing harness. Tested on sbt and maven. Closes #22909 from vanzin/SPARK-25897. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin [SPARK-25957][K8S] Make building alternate language binding docker images optional bin/docker-image-tool.sh tries to build all docker images (JVM, PySpark and SparkR) by default. But not all spark distributions are built with SparkR and hence this script will fail on such distros. With this change, we make building alternate language binding docker images (PySpark and SparkR) optional. User has to specify dockerfile for those language bindings using -p and -R flags accordingly, to build the binding docker images. Tested following scenarios. *bin/docker-image-tool.sh -r -t build* --> Builds only JVM docker image (default behavior) *bin/docker-image-tool.sh -r -t -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile build* --> Builds both JVM and PySpark docker images *bin/docker-image-tool.sh -r -t -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile -R kubernetes/dockerfiles/spark/bindings/R/Dockerfile build* --> Builds JVM, PySpark and SparkR docker images. Author: Nagaram Prasad Addepally Closes #23053 from ramaddepally/SPARK-25957. [SPARK-25960][K8S] Support subpath mounting with Kubernetes 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. 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 Signed-off-by: Marcelo Vanzin [SPARK-26025][K8S] Speed up docker image build on dev repo. [SPARK-26015][K8S] Set a default UID for Spark on K8S Images Adds USER directives to the Dockerfiles which is configurable via build argument (`spark_uid`) for easy customisation. A `-u` flag is added to `bin/docker-image-tool.sh` to make it easy to customise this e.g. ``` > bin/docker-image-tool.sh -r rvesse -t uid -u 185 build > bin/docker-image-tool.sh -r rvesse -t uid push ``` If no UID is explicitly specified it defaults to `185` - this is per skonto's suggestion to align with the OpenShift standard reserved UID for Java apps ( https://lists.openshift.redhat.com/openshift-archives/users/2016-March/msg00283.html) Notes: - We have to make the `WORKDIR` writable by the root group or otherwise jobs will fail with `AccessDeniedException` To Do: - [x] Debug and resolve issue with client mode test - [x] Consider whether to always propagate `SPARK_USER_NAME` to environment of driver and executor pods so `entrypoint.sh` can insert that into `/etc/passwd` entry - [x] Rebase once PR #23013 is merged and update documentation accordingly Built the Docker images with the new Dockerfiles that include the `USER` directives. Ran the Spark on K8S integration tests against the new images. All pass except client mode which I am currently debugging further. Also manually dropped myself into the resulting container images via `docker run` and checked `id -u` output to see that UID is as expected. Tried customising the UID from the default via the new `-u` argument to `docker-image-tool.sh` and again checked the resulting image for the correct runtime UID. cc felixcheung skonto vanzin Closes #23017 from rvesse/SPARK-26015. Authored-by: Rob Vesse Signed-off-by: Marcelo Vanzin [SPARK-25876][K8S] Simplify kubernetes configuration types. [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager. [SPARK-25515][K8S] Adds a config option to keep executor pods for debugging [SPARK-26083][K8S] Add Copy pyspark into corresponding dir cmd in pyspark Dockerfile When I try to run `./bin/pyspark` cmd in a pod in Kubernetes(image built without change from pyspark Dockerfile), I'm getting an error: ``` $SPARK_HOME/bin/pyspark --deploy-mode client --master k8s://https://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT_HTTPS ... Python 2.7.15 (default, Aug 22 2018, 13:24:18) [GCC 6.4.0] on linux2 Type "help", "copyright", "credits" or "license" for more information. Could not open PYTHONSTARTUP IOError: [Errno 2] No such file or directory: '/opt/spark/python/pyspark/shell.py' ``` This is because `pyspark` folder doesn't exist under `/opt/spark/python/` Added `COPY python/pyspark ${SPARK_HOME}/python/pyspark` to pyspark Dockerfile to resolve issue above. Google Kubernetes Engine Closes #23037 from AzureQ/master. Authored-by: Qi Shao Signed-off-by: Marcelo Vanzin [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 Closes #23174 from vanzin/SPARK-26194. [SPARK-25877][K8S] Move all feature logic to feature classes. [SPARK-25922][K8] Spark Driver/Executor "spark-app-selector" label mismatch 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. Manually run." Closes #23322 from suxingfate/SPARK-25922. Lead-authored-by: suxingfate Co-authored-by: xinglwang Signed-off-by: Yinan Li [SPARK-26642][K8S] Add --num-executors option to spark-submit for Spark on K8S. [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 Signed-off-by: Marcelo Vanzin [SPARK-26685][K8S] Correct placement of ARG declaration Latest Docker releases are stricter in their enforcement of build argument scope. The location of the `ARG spark_uid` declaration in the Python and R Dockerfiles means the variable is out of scope by the time it is used in a `USER` declaration resulting in a container running as root rather than the default/configured UID. Also with some of the refactoring of the script that has happened since my PR that introduced the configurable UID it turns out the `-u ` argument is not being properly passed to the Python and R image builds when those are opted into This commit moves the `ARG` declaration to just before the argument is used such that it is in scope. It also ensures that Python and R image builds receive the build arguments that include the `spark_uid` argument where relevant Prior to the patch images are produced where the Python and R images ignore the default/configured UID: ``` > docker run -it --entrypoint /bin/bash rvesse/spark-py:uid456 bash-4.4# whoami root bash-4.4# id -u 0 bash-4.4# exit > docker run -it --entrypoint /bin/bash rvesse/spark:uid456 bash-4.4$ id -u 456 bash-4.4$ exit ``` Note that the Python image is still running as `root` having ignored the configured UID of 456 while the base image has the correct UID because the relevant `ARG` declaration is correctly in scope. After the patch the correct UID is observed: ``` > docker run -it --entrypoint /bin/bash rvesse/spark-r:uid456 bash-4.4$ id -u 456 bash-4.4$ exit exit > docker run -it --entrypoint /bin/bash rvesse/spark-py:uid456 bash-4.4$ id -u 456 bash-4.4$ exit exit > docker run -it --entrypoint /bin/bash rvesse/spark:uid456 bash-4.4$ id -u 456 bash-4.4$ exit ``` Closes #23611 from rvesse/SPARK-26685. Authored-by: Rob Vesse Signed-off-by: Marcelo Vanzin [SPARK-26687][K8S] Fix handling of custom Dockerfile paths With the changes from vanzin's PR #23019 (SPARK-26025) we use a pared down temporary Docker build context which significantly improves build times. However the way this is implemented leads to non-intuitive behaviour when supplying custom Docker file paths. This is because of the following code snippets: ``` (cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[]}" \ -t $(image_ref spark) \ -f "$BASEDOCKERFILE" .) ``` Since the script changes to the temporary build context directory and then runs `docker build` there any path given for the Docker file is taken as relative to the temporary build context directory rather than to the directory where the user invoked the script. This is rather unintuitive and produces somewhat unhelpful errors e.g. ``` > ./bin/docker-image-tool.sh -r rvesse -t badpath -p resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile build Sending build context to Docker daemon 218.4MB Step 1/15 : FROM openjdk:8-alpine ---> 5801f7d008e5 Step 2/15 : ARG spark_uid=185 ---> Using cache ---> 5fd63df1ca39 ... Successfully tagged rvesse/spark:badpath unable to prepare context: unable to evaluate symlinks in Dockerfile path: lstat /Users/rvesse/Documents/Work/Code/spark/target/tmp/docker/pyspark/resource-managers: no such file or directory Failed to build PySpark Docker image, please refer to Docker build output for details. ``` Here we can see that the relative path that was valid where the user typed the command was not valid inside the build context directory. To resolve this we need to ensure that we are resolving relative paths to Docker files appropriately which we do by adding a `resolve_file` function to the script and invoking that on the supplied Docker file paths Validated that relative paths now work as expected: ``` > ./bin/docker-image-tool.sh -r rvesse -t badpath -p resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile build Sending build context to Docker daemon 218.4MB Step 1/15 : FROM openjdk:8-alpine ---> 5801f7d008e5 Step 2/15 : ARG spark_uid=185 ---> Using cache ---> 5fd63df1ca39 Step 3/15 : RUN set -ex && apk upgrade --no-cache && apk add --no-cache bash tini libc6-compat linux-pam krb5 krb5-libs && mkdir -p /opt/spark && mkdir -p /opt/spark/examples && mkdir -p /opt/spark/work-dir && touch /opt/spark/RELEASE && rm /bin/sh && ln -sv /bin/bash /bin/sh && echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && chgrp root /etc/passwd && chmod ug+rw /etc/passwd ---> Using cache ---> eb0a568e032f Step 4/15 : COPY jars /opt/spark/jars ... Successfully tagged rvesse/spark:badpath Sending build context to Docker daemon 6.599MB Step 1/13 : ARG base_img Step 2/13 : ARG spark_uid=185 Step 3/13 : FROM $base_img ---> 8f4fff16f903 Step 4/13 : WORKDIR / ---> Running in 25466e66f27f Removing intermediate container 25466e66f27f ---> 1470b6efae61 Step 5/13 : USER 0 ---> Running in b094b739df37 Removing intermediate container b094b739df37 ---> 6a27eb4acad3 Step 6/13 : RUN mkdir ${SPARK_HOME}/python ---> Running in bc8002c5b17c Removing intermediate container bc8002c5b17c ---> 19bb12f4286a Step 7/13 : RUN apk add --no-cache python && apk add --no-cache python3 && python -m ensurepip && python3 -m ensurepip && rm -r /usr/lib/python*/ensurepip && pip install --upgrade pip setuptools && rm -r /root/.cache ---> Running in 12dcba5e527f ... Successfully tagged rvesse/spark-py:badpath ``` Closes #23613 from rvesse/SPARK-26687. Authored-by: Rob Vesse Signed-off-by: Marcelo Vanzin [SPARK-26794][SQL] SparkSession enableHiveSupport does not point to hive but in-memory while the SparkContext exists ```java public class SqlDemo { public static void main(final String[] args) throws Exception { SparkConf conf = new SparkConf().setAppName("spark-sql-demo"); JavaSparkContext sc = new JavaSparkContext(conf); SparkSession ss = SparkSession.builder().enableHiveSupport().getOrCreate(); ss.sql("show databases").show(); } } ``` Before https://issues.apache.org/jira/browse/SPARK-20946, the demo above point to the right hive metastore if the hive-site.xml is present. But now it can only point to the default in-memory one. Catalog is now as a variable shared across SparkSessions, it is instantiated with SparkContext's conf. After https://issues.apache.org/jira/browse/SPARK-20946, Session level configs are not pass to SparkContext's conf anymore, so the enableHiveSupport API takes no affect on the catalog instance. You can set spark.sql.catalogImplementation=hive application wide to solve the problem, or never create a sc before you call SparkSession.builder().enableHiveSupport().getOrCreate() Here we respect the SparkSession level configuration at the first time to generate catalog within SharedState 1. add ut 2. manually ```scala test("enableHiveSupport has right to determine the catalog while using an existing sc") { val conf = new SparkConf().setMaster("local").setAppName("SharedState Test") val sc = SparkContext.getOrCreate(conf) val ss = SparkSession.builder().enableHiveSupport().getOrCreate() assert(ss.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog], "The catalog should be hive ") val ss2 = SparkSession.builder().getOrCreate() assert(ss2.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog], "The catalog should be shared across sessions") } ``` Without this fix, the above test will fail. You can apply it to `org.apache.spark.sql.hive.HiveSharedStateSuite`, and run, ```sbt ./build/sbt -Phadoop-2.7 -Phive "hive/testOnly org.apache.spark.sql.hive.HiveSharedStateSuite" ``` to verify. Closes #23709 from yaooqinn/SPARK-26794. Authored-by: Kent Yao Signed-off-by: Wenchen Fan [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 Signed-off-by: Marcelo Vanzin [SPARK-25394][CORE] Add an application status metrics source - Exposes several metrics regarding application status as a source, useful to scrape them via jmx instead of mining the metrics rest api. Example use case: prometheus + jmx exporter. - Metrics are gathered when a job ends at the AppStatusListener side, could be more fine-grained but most metrics like tasks completed are also counted by executors. More metrics could be exposed in the future to avoid scraping executors in some scenarios. - a config option `spark.app.status.metrics.enabled` is added to disable/enable these metrics, by default they are disabled. This was manually tested with jmx source enabled and prometheus server on k8s: ![metrics](https://user-images.githubusercontent.com/7945591/45300945-63064d00-b518-11e8-812a-d9b4155ba0c0.png) In the next pic the job delay is shown for repeated pi calculation (Spark action). ![pi](https://user-images.githubusercontent.com/7945591/45329927-89a1a380-b56b-11e8-9cc1-5e76cb83969f.png) Closes #22381 from skonto/add_app_status_metrics. Authored-by: Stavros Kontopoulos Signed-off-by: Marcelo Vanzin [SPARK-25926][CORE] Move config entries in core module to internal.config. [SPARK-26489][CORE] Use ConfigEntry for hardcoded configs for python/r categories [SPARK-26445][CORE] Use ConfigEntry for hardcoded configs for driver/executor categories. [SPARK-20327][CORE][YARN] Add CLI support for YARN custom resources, like GPUs [SPARK-26239] File-based secret key loading for SASL. [SPARK-26482][CORE] Use ConfigEntry for hardcoded configs for ui categories [SPARK-26466][CORE] Use ConfigEntry for hardcoded configs for submit categories. [SPARK-24736][K8S] Let spark-submit handle dependency resolution. [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 Signed-off-by: Dongjoon Hyun [K8S][MINOR] Log minikube version when running integration tests. Closes #23893 from vanzin/minikube-version. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin [SPARK-26995][K8S] Make ld-linux-x86-64.so.2 visible to snappy native library under /lib in docker image with Alpine Linux [SPARK-27023][K8S] Make k8s client timeouts configurable Make k8s client timeouts configurable. No test suite exists for the client factory class, happy to add one if needed Closes #23928 from onursatici/os/k8s-client-timeouts. Lead-authored-by: Onur Satici Co-authored-by: Onur Satici Signed-off-by: Dongjoon Hyun [SPARK-27061][K8S] Expose Driver UI port on driver service to access … Expose Spark UI port on driver service to access logs from service. The patch was tested using unit tests being contributed as a part of the PR Closes #23990 from chandulal/SPARK-27061. Authored-by: chandulal.kavar Signed-off-by: Marcelo Vanzin [SPARK-26343][K8S] Try to speed up running local k8s integration tests Speed up running k8s integration tests locally by allowing folks to skip the tgz dist build and extraction Run tests locally without a distribution of Spark, just a local build Closes #23380 from holdenk/SPARK-26343-Speed-up-running-the-kubernetes-integration-tests-locally. Authored-by: Holden Karau Signed-off-by: Marcelo Vanzin [SPARK-26729][K8S] Make image names under test configurable [SPARK-24793][K8S] Enhance spark-submit for app management - supports `--kill` & `--status` flags. - supports globs which is useful in general check this long standing [issue](https://github.com/kubernetes/kubernetes/issues/17144#issuecomment-272052461) for kubectl. Manually against running apps. Example output: Submission Id reported at launch time: ``` 2019-01-20 23:47:56 INFO Client:58 - Waiting for application spark-pi with submissionId spark:spark-pi-1548020873671-driver to finish... ``` Killing the app: ``` ./bin/spark-submit --kill spark:spark-pi-1548020873671-driver --master k8s://https://192.168.2.8:8443 2019-01-20 23:48:07 WARN Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0) 2019-01-20 23:48:07 WARN Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address ``` App terminates with 143 (SIGTERM, since we have tiny this should lead to [graceful shutdown](https://cloud.google.com/solutions/best-practices-for-building-containers)): ``` 2019-01-20 23:48:08 INFO LoggingPodStatusWatcherImpl:58 - State changed, new state: pod name: spark-pi-1548020873671-driver namespace: spark labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65 creation time: 2019-01-20T21:47:55Z service account name: spark-sa volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm node name: minikube start time: 2019-01-20T21:47:55Z phase: Running container status: container name: spark-kubernetes-driver container image: skonto/spark:k8s-3.0.0 container state: running container started at: 2019-01-20T21:48:00Z 2019-01-20 23:48:09 INFO LoggingPodStatusWatcherImpl:58 - State changed, new state: pod name: spark-pi-1548020873671-driver namespace: spark labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65 creation time: 2019-01-20T21:47:55Z service account name: spark-sa volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm node name: minikube start time: 2019-01-20T21:47:55Z phase: Failed container status: container name: spark-kubernetes-driver container image: skonto/spark:k8s-3.0.0 container state: terminated container started at: 2019-01-20T21:48:00Z container finished at: 2019-01-20T21:48:08Z exit code: 143 termination reason: Error 2019-01-20 23:48:09 INFO LoggingPodStatusWatcherImpl:58 - Container final statuses: container name: spark-kubernetes-driver container image: skonto/spark:k8s-3.0.0 container state: terminated container started at: 2019-01-20T21:48:00Z container finished at: 2019-01-20T21:48:08Z exit code: 143 termination reason: Error 2019-01-20 23:48:09 INFO Client:58 - Application spark-pi with submissionId spark:spark-pi-1548020873671-driver finished. 2019-01-20 23:48:09 INFO ShutdownHookManager:58 - Shutdown hook called 2019-01-20 23:48:09 INFO ShutdownHookManager:58 - Deleting directory /tmp/spark-f114b2e0-5605-4083-9203-a4b1c1f6059e ``` Glob scenario: ``` ./bin/spark-submit --status spark:spark-pi* --master k8s://https://192.168.2.8:8443 2019-01-20 22:27:44 WARN Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0) 2019-01-20 22:27:44 WARN Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address Application status (driver): pod name: spark-pi-1547948600328-driver namespace: spark labels: spark-app-selector -> spark-f13f01702f0b4503975ce98252d59b94, spark-role -> driver pod uid: c576e1c6-1c54-11e9-8215-a434d9270a65 creation time: 2019-01-20T01:43:22Z service account name: spark-sa volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm node name: minikube start time: 2019-01-20T01:43:22Z phase: Running container status: container name: spark-kubernetes-driver container image: skonto/spark:k8s-3.0.0 container state: running container started at: 2019-01-20T01:43:27Z Application status (driver): pod name: spark-pi-1547948792539-driver namespace: spark labels: spark-app-selector -> spark-006d252db9b24f25b5069df357c30264, spark-role -> driver pod uid: 38375b4b-1c55-11e9-8215-a434d9270a65 creation time: 2019-01-20T01:46:35Z service account name: spark-sa volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm node name: minikube start time: 2019-01-20T01:46:35Z phase: Succeeded container status: container name: spark-kubernetes-driver container image: skonto/spark:k8s-3.0.0 container state: terminated container started at: 2019-01-20T01:46:39Z container finished at: 2019-01-20T01:46:56Z exit code: 0 termination reason: Completed ``` Closes #23599 from skonto/submit_ops_extension. Authored-by: Stavros Kontopoulos Signed-off-by: Marcelo Vanzin [SPARK-24902][K8S] Add PV integration tests - Adds persistent volume integration tests - Adds a custom tag to the test to exclude it if it is run against a cloud backend. - Assumes default fs type for the host, AFAIK that is ext4. Manually run the tests against minikube as usual: ``` [INFO] --- scalatest-maven-plugin:1.0:test (integration-test) spark-kubernetes-integration-tests_2.12 --- Discovery starting. Discovery completed in 192 milliseconds. Run starting. Expected test count is: 16 KubernetesSuite: - Run SparkPi with no resources - Run SparkPi with a very long application name. - Use SparkLauncher.NO_RESOURCE - Run SparkPi with a master URL without a scheme. - Run SparkPi with an argument. - Run SparkPi with custom labels, annotations, and environment variables. - Run extraJVMOptions check on driver - Run SparkRemoteFileTest using a remote data file - Run SparkPi with env and mount secrets. - Run PySpark on simple pi.py example - Run PySpark with Python2 to test a pyfiles example - Run PySpark with Python3 to test a pyfiles example - Run PySpark with memory customization - Run in client mode. - Start pod creation from template - Test PVs with local storage ``` Closes #23514 from skonto/pvctests. Authored-by: Stavros Kontopoulos Signed-off-by: shane knapp [SPARK-27216][CORE][BACKPORT-2.4] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue Fix ImplicitCastInputTypes --- R/pkg/R/functions.R | 5 +- bin/docker-image-tool.sh | 207 ++++++++-- .../spark/ExecutorAllocationManager.scala | 4 +- .../org/apache/spark/SecurityManager.scala | 107 +++-- .../scala/org/apache/spark/SparkConf.scala | 59 +-- .../scala/org/apache/spark/SparkContext.scala | 24 +- .../scala/org/apache/spark/SparkEnv.scala | 21 +- .../spark/api/python/PythonRunner.scala | 10 +- .../api/python/PythonWorkerFactory.scala | 35 +- .../org/apache/spark/api/r/RBackend.scala | 10 +- .../apache/spark/api/r/RBackendHandler.scala | 7 +- .../org/apache/spark/api/r/RRunner.scala | 8 +- .../scala/org/apache/spark/api/r/RUtils.scala | 3 +- .../org/apache/spark/deploy/Client.scala | 14 +- .../spark/deploy/FaultToleranceTest.scala | 12 +- .../spark/deploy/LocalSparkCluster.scala | 4 +- .../org/apache/spark/deploy/RRunner.scala | 9 +- .../spark/deploy/SparkCuratorUtil.scala | 3 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 14 - .../org/apache/spark/deploy/SparkSubmit.scala | 134 ++++-- .../spark/deploy/SparkSubmitArguments.scala | 53 ++- .../deploy/history/FsHistoryProvider.scala | 50 ++- .../spark/deploy/history/HistoryServer.scala | 12 +- .../history/HistoryServerDiskManager.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 58 +-- .../spark/deploy/master/MasterArguments.scala | 5 +- .../deploy/master/RecoveryModeFactory.scala | 7 +- .../master/ZooKeeperLeaderElectionAgent.scala | 5 +- .../master/ZooKeeperPersistenceEngine.scala | 15 +- .../spark/deploy/master/ui/MasterWebUI.scala | 3 +- .../deploy/rest/RestSubmissionClient.scala | 9 +- .../deploy/rest/StandaloneRestServer.scala | 23 +- .../rest/SubmitRestProtocolRequest.scala | 11 +- .../HadoopDelegationTokenManager.scala | 278 ++++++++++--- .../HadoopFSDelegationTokenProvider.scala | 5 +- .../spark/deploy/worker/DriverRunner.scala | 6 +- .../spark/deploy/worker/DriverWrapper.scala | 6 +- .../spark/deploy/worker/ExecutorRunner.scala | 3 +- .../apache/spark/deploy/worker/Worker.scala | 23 +- .../spark/deploy/worker/WorkerArguments.scala | 5 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 - .../apache/spark/internal/config/Deploy.scala | 68 +++ .../config/History.scala} | 45 +- .../apache/spark/internal/config/Kryo.scala | 57 +++ .../apache/spark/internal/config/Python.scala | 47 +++ .../org/apache/spark/internal/config/R.scala | 40 ++ .../config/Status.scala} | 12 +- .../apache/spark/internal/config/Tests.scala | 56 +++ .../org/apache/spark/internal/config/UI.scala | 145 +++++++ .../apache/spark/internal/config/Worker.scala | 63 +++ .../spark/internal/config/package.scala | 287 +++++++++++-- .../spark/memory/StaticMemoryManager.scala | 9 +- .../spark/memory/UnifiedMemoryManager.scala | 9 +- .../apache/spark/metrics/MetricsConfig.scala | 3 +- .../apache/spark/metrics/MetricsSystem.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 39 +- .../cluster/StandaloneSchedulerBackend.scala | 16 +- .../local/LocalSchedulerBackend.scala | 4 +- .../spark/serializer/JavaSerializer.scala | 5 +- .../spark/serializer/KryoSerializer.scala | 30 +- .../spark/status/AppStatusListener.scala | 36 +- .../apache/spark/status/AppStatusSource.scala | 78 ++++ .../apache/spark/status/AppStatusStore.scala | 7 +- .../spark/status/ElementTrackingStore.scala | 3 +- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../apache/spark/ui/ConsoleProgressBar.scala | 4 +- .../apache/spark/ui/HttpSecurityFilter.scala | 116 ++++++ .../org/apache/spark/ui/JettyUtils.scala | 44 +- .../scala/org/apache/spark/ui/SparkUI.scala | 6 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../apache/spark/util/CommandLineUtils.scala | 8 +- .../org/apache/spark/util/RpcUtils.scala | 5 +- .../scala/org/apache/spark/util/Utils.scala | 19 +- .../spark/util/logging/FileAppender.scala | 10 +- .../util/logging/RollingFileAppender.scala | 19 +- .../org/apache/spark/CheckpointSuite.scala | 3 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../apache/spark/JobCancellationSuite.scala | 3 +- .../apache/spark/SecurityManagerSuite.scala | 281 ++++++++----- .../scala/org/apache/spark/ShuffleSuite.scala | 4 +- .../org/apache/spark/SparkConfSuite.scala | 40 +- .../org/apache/spark/SparkContextSuite.scala | 4 + .../api/python/PythonBroadcastSuite.scala | 3 +- .../apache/spark/benchmark}/Benchmark.scala | 105 ++--- .../spark/benchmark/BenchmarkBase.scala | 69 ++++ .../spark/broadcast/BroadcastSuite.scala | 6 +- .../spark/deploy/SparkSubmitSuite.scala | 301 +++++++------- .../StandaloneDynamicAllocationSuite.scala | 6 +- .../history/FsHistoryProviderSuite.scala | 25 +- .../HistoryServerDiskManagerSuite.scala | 3 +- .../deploy/history/HistoryServerSuite.scala | 10 +- .../spark/deploy/master/MasterSuite.scala | 24 +- .../master/PersistenceEngineSuite.scala | 3 +- .../deploy/master/ui/MasterWebUISuite.scala | 7 +- .../deploy/rest/SubmitRestProtocolSuite.scala | 3 +- .../HadoopDelegationTokenManagerSuite.scala | 142 ++----- .../spark/deploy/worker/WorkerSuite.scala | 9 +- .../apache/spark/executor/ExecutorSuite.scala | 5 +- .../spark/launcher/LauncherBackendSuite.scala | 3 +- .../memory/UnifiedMemoryManagerSuite.scala | 2 +- .../spark/metrics/MetricsConfigSuite.scala | 9 +- .../spark/metrics/MetricsSystemSuite.scala | 2 +- .../NettyBlockTransferSecuritySuite.scala | 27 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 29 +- .../spark/scheduler/MapStatusSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 3 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../GenericAvroSerializerSuite.scala | 3 +- .../spark/serializer/KryoBenchmark.scala | 11 +- .../serializer/KryoSerializerBenchmark.scala | 93 +++++ .../KryoSerializerDistributedSuite.scala | 4 +- .../KryoSerializerResizableOutputSuite.scala | 14 +- .../serializer/KryoSerializerSuite.scala | 138 ++++--- .../SerializerPropertiesSuite.scala | 3 +- .../UnsafeKryoSerializerSuite.scala | 6 +- .../spark/status/AppStatusListenerSuite.scala | 3 +- .../status/ElementTrackingStoreSuite.scala | 3 +- .../BlockManagerReplicationSuite.scala | 8 +- .../spark/storage/BlockManagerSuite.scala | 2 +- .../spark/storage/FlatmapIteratorSuite.scala | 4 +- .../org/apache/spark/ui/StagePageSuite.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 15 +- .../scala/org/apache/spark/ui/UISuite.scala | 3 +- .../apache/spark/util/FileAppenderSuite.scala | 13 +- .../org/apache/spark/util/UtilsSuite.scala | 3 +- .../ExternalAppendOnlyMapSuite.scala | 4 +- .../util/collection/ExternalSorterSuite.scala | 8 +- dev/deps/spark-deps-hadoop-2.6 | 3 +- dev/deps/spark-deps-hadoop-2.7 | 5 +- dev/deps/spark-deps-hadoop-3.1 | 5 +- docs/running-on-kubernetes.md | 387 +++++++++++++++++- docs/running-on-yarn.md | 37 ++ docs/security.md | 147 ++++++- .../spark/examples/DFSReadWriteTest.scala | 12 +- .../org/apache/spark/examples/HdfsTest.scala | 2 + .../flume/FlumePollingStreamSuite.scala | 11 +- .../sql/kafka010/KafkaRelationSuite.scala | 9 +- .../spark/sql/kafka010/KafkaSinkSuite.scala | 9 +- .../kafka010/DirectKafkaStreamSuite.scala | 11 +- .../streaming/kafka010/KafkaRDDSuite.scala | 23 +- .../kafka/DirectKafkaStreamSuite.scala | 11 +- .../streaming/kafka/KafkaClusterSuite.scala | 11 +- .../spark/streaming/kafka/KafkaRDDSuite.scala | 23 +- .../streaming/kafka/KafkaStreamSuite.scala | 23 +- .../kafka/ReliableKafkaStreamSuite.scala | 13 +- .../KinesisInputDStreamBuilderSuite.scala | 6 +- .../kinesis/KinesisStreamSuite.scala | 26 +- .../spark/ml/attribute/AttributeSuite.scala | 19 + .../spark/ml/feature/InstanceSuite.scala | 3 +- .../spark/ml/feature/LabeledPointSuite.scala | 3 +- .../spark/ml/tree/impl/TreePointSuite.scala | 3 +- .../spark/mllib/clustering/KMeansSuite.scala | 3 +- .../spark/mllib/feature/Word2VecSuite.scala | 15 +- .../spark/mllib/linalg/MatricesSuite.scala | 3 +- .../linalg/UDTSerializationBenchmark.scala | 2 +- .../spark/mllib/linalg/VectorsSuite.scala | 3 +- .../mllib/regression/LabeledPointSuite.scala | 3 +- .../MultivariateGaussianSuite.scala | 2 + pom.xml | 14 +- project/SparkBuild.scala | 62 +++ python/pyspark/sql/functions.py | 4 +- resource-managers/kubernetes/core/pom.xml | 6 + ...g.apache.spark.deploy.SparkSubmitOperation | 1 + .../org/apache/spark/deploy/k8s/Config.scala | 166 ++++++-- .../apache/spark/deploy/k8s/Constants.scala | 48 ++- .../spark/deploy/k8s/KubernetesConf.scala | 313 ++++++-------- .../deploy/k8s/KubernetesDriverSpec.scala | 7 - .../spark/deploy/k8s/KubernetesUtils.scala | 183 ++++++++- .../deploy/k8s/KubernetesVolumeSpec.scala | 11 +- .../deploy/k8s/KubernetesVolumeUtils.scala | 55 +-- .../k8s/SparkKubernetesClientFactory.scala | 36 +- .../k8s/features/BasicDriverFeatureStep.scala | 65 +-- .../features/BasicExecutorFeatureStep.scala | 156 +++---- .../features/DriverCommandFeatureStep.scala | 98 +++++ ...iverKubernetesCredentialsFeatureStep.scala | 6 +- .../features/DriverServiceFeatureStep.scala | 35 +- .../k8s/features/EnvSecretsFeatureStep.scala | 11 +- .../HadoopConfExecutorFeatureStep.scala | 40 ++ .../HadoopSparkUserExecutorFeatureStep.scala} | 31 +- .../KerberosConfDriverFeatureStep.scala | 206 ++++++++++ .../KerberosConfExecutorFeatureStep.scala | 46 +++ .../KubernetesFeatureConfigStep.scala | 4 +- .../k8s/features/LocalDirsFeatureStep.scala | 12 +- .../features/MountSecretsFeatureStep.scala | 13 +- .../features/MountVolumesFeatureStep.scala | 13 +- .../features/PodTemplateConfigMapStep.scala | 87 ++++ .../bindings/JavaDriverFeatureStep.scala | 46 --- .../bindings/PythonDriverFeatureStep.scala | 76 ---- .../bindings/RDriverFeatureStep.scala | 62 --- .../hadooputils/HadoopBootstrapUtil.scala | 283 +++++++++++++ .../hadooputils/HadoopKerberosLogin.scala | 64 +++ .../hadooputils/KerberosConfigSpec.scala | 33 ++ ...bernetesHadoopDelegationTokenManager.scala | 37 ++ .../deploy/k8s/submit/K8sSubmitOps.scala | 188 +++++++++ .../submit/KubernetesClientApplication.scala | 78 ++-- .../k8s/submit/KubernetesDriverBuilder.scala | 100 ++--- .../k8s/submit/LoggingPodStatusWatcher.scala | 77 +--- .../deploy/k8s/submit/MainAppResource.scala | 3 +- .../cluster/k8s/ExecutorPodsAllocator.scala | 25 +- .../k8s/ExecutorPodsLifecycleManager.scala | 18 +- .../k8s/KubernetesClusterManager.scala | 18 +- .../KubernetesClusterSchedulerBackend.scala | 21 +- .../k8s/KubernetesExecutorBuilder.scala | 71 ++-- .../deploy/k8s/KubernetesConfSuite.scala | 164 +------- .../spark/deploy/k8s/KubernetesTestConf.scala | 137 +++++++ .../deploy/k8s/KubernetesUtilsSuite.scala | 68 +++ .../k8s/KubernetesVolumeUtilsSuite.scala | 40 +- .../spark/deploy/k8s/PodBuilderSuite.scala | 177 ++++++++ .../BasicDriverFeatureStepSuite.scala | 179 +++----- .../BasicExecutorFeatureStepSuite.scala | 178 ++++---- .../DriverCommandFeatureStepSuite.scala | 93 +++++ ...ubernetesCredentialsFeatureStepSuite.scala | 69 +--- .../DriverServiceFeatureStepSuite.scala | 254 +++++------- .../features/EnvSecretsFeatureStepSuite.scala | 32 +- .../features/LocalDirsFeatureStepSuite.scala | 69 ++-- .../MountSecretsFeatureStepSuite.scala | 21 +- .../MountVolumesFeatureStepSuite.scala | 110 +++-- .../PodTemplateConfigMapStepSuite.scala | 86 ++++ .../bindings/JavaDriverFeatureStepSuite.scala | 60 --- .../PythonDriverFeatureStepSuite.scala | 110 ----- .../bindings/RDriverFeatureStepSuite.scala | 63 --- .../spark/deploy/k8s/submit/ClientSuite.scala | 48 +-- .../deploy/k8s/submit/K8sSubmitOpSuite.scala | 156 +++++++ .../submit/KubernetesDriverBuilderSuite.scala | 239 +---------- .../k8s/ExecutorPodsAllocatorSuite.scala | 51 +-- .../ExecutorPodsLifecycleManagerSuite.scala | 27 +- ...bernetesClusterSchedulerBackendSuite.scala | 9 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 106 +---- .../src/main/dockerfiles/spark/Dockerfile | 18 +- .../dockerfiles/spark/bindings/R/Dockerfile | 9 + .../spark/bindings/python/Dockerfile | 11 +- .../src/main/dockerfiles/spark/entrypoint.sh | 36 +- .../kubernetes/integration-tests/README.md | 216 +++++++++- .../dev/dev-run-integration-tests.sh | 43 +- .../kubernetes/integration-tests/pom.xml | 19 +- .../scripts/setup-integration-test-env.sh | 71 +++- .../src/test/resources/driver-template.yml | 26 ++ .../src/test/resources/executor-template.yml | 25 ++ .../k8s/integrationtest/BasicTestsSuite.scala | 3 + .../ClientModeTestsSuite.scala | 3 +- .../k8s/integrationtest/KubernetesSuite.scala | 126 ++++-- .../KubernetesTestComponents.scala | 15 +- .../k8s/integrationtest/PVTestsSuite.scala | 189 +++++++++ .../integrationtest/PodTemplateSuite.scala | 55 +++ .../k8s/integrationtest/ProcessUtils.scala | 9 +- .../integrationtest/PythonTestsSuite.scala | 12 +- .../k8s/integrationtest/RTestsSuite.scala | 5 +- .../integrationtest/SecretsTestsSuite.scala | 28 +- .../k8s/integrationtest/TestConstants.scala | 18 +- .../deploy/k8s/integrationtest/Utils.scala | 22 + .../backend/IntegrationTestBackend.scala | 21 +- .../backend/cloud/KubeConfigBackend.scala | 71 ++++ .../docker/DockerForDesktopBackend.scala | 15 +- .../minikube/MinikubeTestBackend.scala | 1 + .../deploy/mesos/MesosClusterDispatcher.scala | 1 + .../apache/spark/deploy/mesos/config.scala | 15 +- .../deploy/rest/mesos/MesosRestServer.scala | 13 +- .../mesos/MesosClusterPersistenceEngine.scala | 11 +- .../cluster/mesos/MesosClusterScheduler.scala | 26 +- .../MesosCoarseGrainedSchedulerBackend.scala | 38 +- .../MesosFineGrainedSchedulerBackend.scala | 12 +- .../MesosHadoopDelegationTokenManager.scala | 160 -------- .../cluster/mesos/MesosSchedulerUtils.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 35 +- .../org/apache/spark/deploy/yarn/Client.scala | 22 +- .../deploy/yarn/ResourceRequestHelper.scala | 157 +++++++ .../spark/deploy/yarn/YarnAllocator.scala | 39 +- .../org/apache/spark/deploy/yarn/config.scala | 18 +- .../yarn/security/AMCredentialRenewer.scala | 185 --------- .../YARNHadoopDelegationTokenManager.scala | 48 +-- .../cluster/YarnClientSchedulerBackend.scala | 8 +- .../cluster/YarnSchedulerBackend.scala | 43 +- .../spark/deploy/yarn/ClientSuite.scala | 71 ++++ .../yarn/ResourceRequestHelperSuite.scala | 225 ++++++++++ .../yarn/ResourceRequestTestHelper.scala | 89 ++++ .../deploy/yarn/YarnAllocatorSuite.scala | 25 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 6 +- .../yarn/YarnSparkHadoopUtilSuite.scala | 10 +- ...ARNHadoopDelegationTokenManagerSuite.scala | 5 +- sql/catalyst/pom.xml | 4 + .../catalyst/analysis/FunctionRegistry.scala | 18 +- .../catalog/ExternalCatalogUtils.scala | 6 +- .../aggregate/gio/BucketBitmapMerge2.scala | 86 ++++ .../aggregate/gio/CBitmapMerge2.scala | 85 ++++ .../aggregate/gio/CollectBucketBitmap.scala | 92 +++++ .../aggregate/gio/CollectCBitmap.scala | 93 +++++ .../aggregate/gio/CollectSBitmap.scala | 94 +++++ .../aggregate/gio/MergeBucketBitmap.scala | 89 ++++ .../aggregate/gio/MergeCBitmap.scala | 89 ++++ .../aggregate/gio/MergeSBitmap.scala | 89 ++++ .../aggregate/gio/SBitmapMerge2.scala | 85 ++++ .../expressions/aggregate/gio/package.scala | 27 ++ .../spark/sql/catalyst/json/JSONOptions.scala | 5 + .../sql/catalyst/json/JacksonGenerator.scala | 5 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 33 +- .../spark/sql/catalyst/util/BitMapUtils.scala | 84 ++++ .../apache/spark/sql/internal/SQLConf.scala | 24 ++ .../org/apache/spark/sql/HashBenchmark.scala | 2 +- .../spark/sql/HashByteArrayBenchmark.scala | 2 +- .../spark/sql/UnsafeProjectionBenchmark.scala | 2 +- .../optimizer/OptimizerLoggingSuite.scala | 148 +++++++ .../org/apache/spark/sql/SparkSession.scala | 2 +- .../execution/ui/SQLAppStatusListener.scala | 2 +- .../org/apache/spark/sql/functions.scala | 35 ++ .../spark/sql/internal/SharedState.scala | 37 +- .../apache/spark/sql/DatasetBenchmark.scala | 2 +- .../DatasetSerializerRegistratorSuite.scala | 3 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 21 + .../apache/spark/sql/SessionStateSuite.scala | 15 +- .../spark/sql/SparkSessionBuilderSuite.scala | 3 +- .../execution/ExchangeCoordinatorSuite.scala | 17 +- ...nalAppendOnlyUnsafeRowArrayBenchmark.scala | 33 +- ...xternalAppendOnlyUnsafeRowArraySuite.scala | 6 +- .../sql/execution/SparkSqlParserSuite.scala | 6 +- .../SortBasedAggregationStoreSuite.scala | 6 +- .../benchmark/AggregateBenchmark.scala | 5 +- .../execution/benchmark/BenchmarkBase.scala | 54 --- .../benchmark/BenchmarkWideTable.scala | 5 +- .../benchmark/DataSourceReadBenchmark.scala | 7 +- .../benchmark/DataSourceWriteBenchmark.scala | 2 +- .../benchmark/FilterPushdownBenchmark.scala | 8 +- .../execution/benchmark/JoinBenchmark.scala | 4 +- .../execution/benchmark/MiscBenchmark.scala | 5 +- .../benchmark/PrimitiveArrayBenchmark.scala | 5 +- .../execution/benchmark/SortBenchmark.scala | 4 +- .../benchmark/SqlBasedBenchmark.scala | 60 +++ .../benchmark/TPCDSQueryBenchmark.scala | 2 +- .../benchmark/UnsafeArrayDataBenchmark.scala | 4 +- .../benchmark/WideSchemaBenchmark.scala | 10 +- .../CompressionSchemeBenchmark.scala | 2 +- .../BasicWriteTaskStatsTrackerSuite.scala | 6 +- .../datasources/csv/CSVBenchmarks.scala | 3 +- .../datasources/json/JsonBenchmarks.scala | 3 +- .../execution/joins/BroadcastJoinSuite.scala | 8 +- .../execution/joins/HashedRelationSuite.scala | 5 +- .../python/BatchEvalPythonExecSuite.scala | 7 +- .../streaming/state/StateStoreRDDSuite.scala | 7 +- .../ui/SQLAppStatusListenerSuite.scala | 2 +- .../vectorized/ColumnarBatchBenchmark.scala | 4 +- .../internal/ExecutorSideSQLConfSuite.scala | 8 +- .../FlatMapGroupsWithStateSuite.scala | 8 +- .../spark/sql/streaming/StreamTest.scala | 7 +- .../streaming/StreamingAggregationSuite.scala | 8 +- .../StreamingDeduplicationSuite.scala | 7 +- .../spark/sql/test/SharedSQLContext.scala | 7 +- .../hive/thriftserver/HiveThriftServer2.scala | 5 +- .../hive/thriftserver/UISeleniumSuite.scala | 9 +- .../apache/spark/sql/hive/test/TestHive.scala | 6 +- .../sql/catalyst/ExpressionToSQLSuite.scala | 305 ++++++++++++++ .../ObjectHashAggregateExecBenchmark.scala | 4 +- .../HiveExternalCatalogVersionsSuite.scala | 11 +- .../spark/sql/hive/HiveSharedStateSuite.scala | 66 +++ .../spark/sql/hive/HiveSparkSubmitSuite.scala | 23 +- .../hive/execution/ConcurrentHiveSuite.scala | 3 +- .../sql/hive/execution/HiveQuerySuite.scala | 6 + .../sql/hive/execution/HiveUDAFSuite.scala | 1 + .../execution/ObjectHashAggregateSuite.scala | 1 + .../spark/sql/hive/orc/OrcReadBenchmark.scala | 3 +- .../apache/spark/streaming/Checkpoint.scala | 5 +- .../spark/streaming/StreamingContext.scala | 3 +- .../spark/streaming/ReceiverSuite.scala | 3 +- .../spark/streaming/UISeleniumSuite.scala | 3 +- 365 files changed, 10422 insertions(+), 4344 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Deploy.scala rename core/src/main/scala/org/apache/spark/{deploy/history/config.scala => internal/config/History.scala} (65%) create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Kryo.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Python.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/R.scala rename core/src/main/scala/org/apache/spark/{status/config.scala => internal/config/Status.scala} (83%) create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Tests.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/UI.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/Worker.scala create mode 100644 core/src/main/scala/org/apache/spark/status/AppStatusSource.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala rename core/src/{main/scala/org/apache/spark/util => test/scala/org/apache/spark/benchmark}/Benchmark.scala (67%) create mode 100644 core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala create mode 100644 resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.SparkSubmitOperation create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala rename resource-managers/kubernetes/{integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala => core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala} (50%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOpSuite.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala rename core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala => resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala (68%) delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala delete mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala create mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala create mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/BucketBitmapMerge2.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/CBitmapMerge2.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/CollectBucketBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/CollectCBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/CollectSBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/MergeBucketBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/MergeCBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/MergeSBitmap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/SBitmapMerge2.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/gio/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BitMapUtils.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 1e702446539e3..71d8a72e04fe6 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -198,8 +198,9 @@ NULL #' } #' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains #' additional named properties to control how it is converted, accepts the same -#' options as the JSON data source. In \code{arrays_zip}, this contains additional -#' Columns of arrays to be merged. +#' options as the JSON data source. Additionally \code{to_json} supports the "pretty" +#' option which enables pretty JSON generation. In \code{arrays_zip}, this contains +#' additional Columns of arrays to be merged. #' @name column_collection_functions #' @rdname column_collection_functions #' @family collection functions diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 5e8eafff50f2a..0388e23979dda 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -29,6 +29,20 @@ if [ -z "${SPARK_HOME}" ]; then fi . "${SPARK_HOME}/bin/load-spark-env.sh" +CTX_DIR="$SPARK_HOME/target/tmp/docker" + +function is_dev_build { + [ ! -f "$SPARK_HOME/RELEASE" ] +} + +function cleanup_ctx_dir { + if is_dev_build; then + rm -rf "$CTX_DIR" + fi +} + +trap cleanup_ctx_dir EXIT + function image_ref { local image="$1" local add_repo="${2:-1}" @@ -41,57 +55,154 @@ function image_ref { echo "$image" } +function docker_push { + local image_name="$1" + if [ ! -z $(docker images -q "$(image_ref ${image_name})") ]; then + docker push "$(image_ref ${image_name})" + if [ $? -ne 0 ]; then + error "Failed to push $image_name Docker image." + fi + else + echo "$(image_ref ${image_name}) image not found. Skipping push for this image." + fi +} + +function resolve_file { + local FILE=$1 + if [ -n "$FILE" ]; then + local DIR=$(dirname $FILE) + DIR=$(cd $DIR && pwd) + FILE="${DIR}/$(basename $FILE)" + fi + echo $FILE +} + +# Create a smaller build context for docker in dev builds to make the build faster. Docker +# uploads all of the current directory to the daemon, and it can get pretty big with dev +# builds that contain test log files and other artifacts. +# +# Three build contexts are created, one for each image: base, pyspark, and sparkr. For them +# to have the desired effect, the docker command needs to be executed inside the appropriate +# context directory. +# +# Note: docker does not support symlinks in the build context. +function create_dev_build_context {( + set -e + local BASE_CTX="$CTX_DIR/base" + mkdir -p "$BASE_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$BASE_CTX/kubernetes/dockerfiles" + + cp -r "assembly/target/scala-$SPARK_SCALA_VERSION/jars" "$BASE_CTX/jars" + cp -r "resource-managers/kubernetes/integration-tests/tests" \ + "$BASE_CTX/kubernetes/tests" + + mkdir "$BASE_CTX/examples" + cp -r "examples/src" "$BASE_CTX/examples/src" + # Copy just needed examples jars instead of everything. + mkdir "$BASE_CTX/examples/jars" + for i in examples/target/scala-$SPARK_SCALA_VERSION/jars/*; do + if [ ! -f "$BASE_CTX/jars/$(basename $i)" ]; then + cp $i "$BASE_CTX/examples/jars" + fi + done + + for other in bin sbin data; do + cp -r "$other" "$BASE_CTX/$other" + done + + local PYSPARK_CTX="$CTX_DIR/pyspark" + mkdir -p "$PYSPARK_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$PYSPARK_CTX/kubernetes/dockerfiles" + mkdir "$PYSPARK_CTX/python" + cp -r "python/lib" "$PYSPARK_CTX/python/lib" + cp -r "python/pyspark" "$PYSPARK_CTX/python/pyspark" + + local R_CTX="$CTX_DIR/sparkr" + mkdir -p "$R_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$R_CTX/kubernetes/dockerfiles" + cp -r "R" "$R_CTX/R" +)} + +function img_ctx_dir { + if is_dev_build; then + echo "$CTX_DIR/$1" + else + echo "$SPARK_HOME" + fi +} + function build { local BUILD_ARGS - local IMG_PATH - - if [ ! -f "$SPARK_HOME/RELEASE" ]; then - # Set image build arguments accordingly if this is a source repo and not a distribution archive. - IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles - BUILD_ARGS=( - ${BUILD_PARAMS} - --build-arg - img_path=$IMG_PATH - --build-arg - spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars - --build-arg - k8s_tests=resource-managers/kubernetes/integration-tests/tests - ) - else - # Not passed as an argument to docker, but used to validate the Spark directory. - IMG_PATH="kubernetes/dockerfiles" - BUILD_ARGS=(${BUILD_PARAMS}) + local SPARK_ROOT="$SPARK_HOME" + + if is_dev_build; then + create_dev_build_context || error "Failed to create docker build context." + SPARK_ROOT="$CTX_DIR/base" fi - if [ ! -d "$IMG_PATH" ]; then + # Verify that the Docker image content directory is present + if [ ! -d "$SPARK_ROOT/kubernetes/dockerfiles" ]; then error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark." fi + + # Verify that Spark has actually been built/is a runnable distribution + # i.e. the Spark JARs that the Docker files will place into the image are present + local TOTAL_JARS=$(ls $SPARK_ROOT/jars/spark-* | wc -l) + TOTAL_JARS=$(( $TOTAL_JARS )) + if [ "${TOTAL_JARS}" -eq 0 ]; then + error "Cannot find Spark JARs. This script assumes that Apache Spark has first been built locally or this is a runnable distribution." + fi + + local BUILD_ARGS=(${BUILD_PARAMS}) + + # If a custom SPARK_UID was set add it to build arguments + if [ -n "$SPARK_UID" ]; then + BUILD_ARGS+=(--build-arg spark_uid=$SPARK_UID) + fi + local BINDING_BUILD_ARGS=( - ${BUILD_PARAMS} + ${BUILD_ARGS[@]} --build-arg base_img=$(image_ref spark) ) - local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"} - local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"} - local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"} - docker build $NOCACHEARG "${BUILD_ARGS[@]}" \ + local BASEDOCKERFILE=${BASEDOCKERFILE:-"kubernetes/dockerfiles/spark/Dockerfile"} + local PYDOCKERFILE=${PYDOCKERFILE:-false} + local RDOCKERFILE=${RDOCKERFILE:-false} + + (cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[@]}" \ -t $(image_ref spark) \ - -f "$BASEDOCKERFILE" . + -f "$BASEDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build Spark JVM Docker image, please refer to Docker build output for details." + fi - docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ - -t $(image_ref spark-py) \ - -f "$PYDOCKERFILE" . + if [ "${PYDOCKERFILE}" != "false" ]; then + (cd $(img_ctx_dir pyspark) && docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ + -t $(image_ref spark-py) \ + -f "$PYDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build PySpark Docker image, please refer to Docker build output for details." + fi + fi - docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ - -t $(image_ref spark-r) \ - -f "$RDOCKERFILE" . + if [ "${RDOCKERFILE}" != "false" ]; then + (cd $(img_ctx_dir sparkr) && docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ + -t $(image_ref spark-r) \ + -f "$RDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build SparkR Docker image, please refer to Docker build output for details." + fi + fi } function push { - docker push "$(image_ref spark)" - docker push "$(image_ref spark-py)" - docker push "$(image_ref spark-r)" + docker_push "spark" + docker_push "spark-py" + docker_push "spark-r" } function usage { @@ -106,14 +217,18 @@ Commands: Options: -f file Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark. - -p file Dockerfile to build for PySpark Jobs. Builds Python dependencies and ships with Spark. - -R file Dockerfile to build for SparkR Jobs. Builds R dependencies and ships with Spark. + -p file (Optional) Dockerfile to build for PySpark Jobs. Builds Python dependencies and ships with Spark. + Skips building PySpark docker image if not specified. + -R file (Optional) Dockerfile to build for SparkR Jobs. Builds R dependencies and ships with Spark. + Skips building SparkR docker image if not specified. -r repo Repository address. -t tag Tag to apply to the built image, or to identify the image to be pushed. -m Use minikube's Docker daemon. -n Build docker image with --no-cache - -b arg Build arg to build or push the image. For multiple build args, this option needs to - be used separately for each build arg. + -u uid UID to use in the USER directive to set the user the main Spark process runs as inside the + resulting container + -b arg Build arg to build or push the image. For multiple build args, this option needs to + be used separately for each build arg. Using minikube when building images will do so directly into minikube's Docker daemon. There is no need to push the images into minikube in that case, they'll be automatically @@ -127,6 +242,9 @@ Examples: - Build image in minikube with tag "testing" $0 -m -t testing build + - Build PySpark docker image + $0 -r docker.io/myrepo -t v2.3.0 -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile build + - Build and push image with tag "v2.3.0" to docker.io/myrepo $0 -r docker.io/myrepo -t v2.3.0 build $0 -r docker.io/myrepo -t v2.3.0 push @@ -145,13 +263,14 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:nb: option +SPARK_UID= +while getopts f:p:R:mr:t:nb:u: option do case "${option}" in - f) BASEDOCKERFILE=${OPTARG};; - p) PYDOCKERFILE=${OPTARG};; - R) RDOCKERFILE=${OPTARG};; + f) BASEDOCKERFILE=$(resolve_file ${OPTARG});; + p) PYDOCKERFILE=$(resolve_file ${OPTARG});; + R) RDOCKERFILE=$(resolve_file ${OPTARG});; r) REPO=${OPTARG};; t) TAG=${OPTARG};; n) NOCACHEARG="--no-cache";; @@ -160,8 +279,12 @@ do if ! which minikube 1>/dev/null; then error "Cannot find minikube." fi + if ! minikube status 1>/dev/null; then + error "Cannot contact minikube. Make sure it's running." + fi eval $(minikube docker-env) ;; + u) SPARK_UID=${OPTARG};; esac done diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 36819aacb4ff9..6d5466e48d876 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -121,7 +121,7 @@ private[spark] class ExecutorAllocationManager( // allocation is only supported for YARN and the default number of cores per executor in YARN is // 1, but it might need to be attained differently for different cluster managers private val tasksPerExecutorForFullParallelism = - conf.getInt("spark.executor.cores", 1) / conf.getInt("spark.task.cpus", 1) + conf.get(EXECUTOR_CORES) / conf.getInt("spark.task.cpus", 1) private val executorAllocationRatio = conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO) @@ -217,7 +217,7 @@ private[spark] class ExecutorAllocationManager( "shuffle service. You may enable this through spark.shuffle.service.enabled.") } if (tasksPerExecutorForFullParallelism == 0) { - throw new SparkException("spark.executor.cores must not be < spark.task.cpus.") + throw new SparkException(s"${EXECUTOR_CORES.key} must not be < spark.task.cpus.") } if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) { diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3cfafeb951105..c64fdc02efc73 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -17,8 +17,11 @@ package org.apache.spark +import java.io.File import java.net.{Authenticator, PasswordAuthentication} import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files +import java.util.Base64 import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -26,6 +29,7 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.sasl.SecretKeyHolder import org.apache.spark.util.Utils @@ -43,7 +47,8 @@ import org.apache.spark.util.Utils */ private[spark] class SecurityManager( sparkConf: SparkConf, - val ioEncryptionKey: Option[Array[Byte]] = None) + val ioEncryptionKey: Option[Array[Byte]] = None, + authSecretFileConf: ConfigEntry[Option[String]] = AUTH_SECRET_FILE) extends Logging with SecretKeyHolder { import SecurityManager._ @@ -52,17 +57,13 @@ private[spark] class SecurityManager( private val WILDCARD_ACL = "*" private val authOn = sparkConf.get(NETWORK_AUTH_ENABLED) - // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = - sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) + private var aclsOn = sparkConf.get(ACLS_ENABLE) // admin acls should be set before view or modify acls - private var adminAcls: Set[String] = - stringToSet(sparkConf.get("spark.admin.acls", "")) + private var adminAcls: Set[String] = sparkConf.get(ADMIN_ACLS).toSet // admin group acls should be set before view or modify group acls - private var adminAclsGroups : Set[String] = - stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var adminAclsGroups: Set[String] = sparkConf.get(ADMIN_ACLS_GROUPS).toSet private var viewAcls: Set[String] = _ @@ -78,11 +79,11 @@ private[spark] class SecurityManager( private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) - setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) - setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAcls(defaultAclUsers, sparkConf.get(UI_VIEW_ACLS)) + setModifyAcls(defaultAclUsers, sparkConf.get(MODIFY_ACLS)) - setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); - setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + setViewAclsGroups(sparkConf.get(UI_VIEW_ACLS_GROUPS)) + setModifyAclsGroups(sparkConf.get(MODIFY_ACLS_GROUPS)) private var secretKey: String = _ logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + @@ -123,23 +124,16 @@ private[spark] class SecurityManager( opts } - /** - * Split a comma separated String, filter out any empty items, and return a Set of strings - */ - private def stringToSet(list: String): Set[String] = { - list.split(',').map(_.trim).filter(!_.isEmpty).toSet - } - /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setViewAcls(defaultUsers: Set[String], allowedUsers: String) { - viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setViewAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + viewAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing view acls to: " + viewAcls.mkString(",")) } - def setViewAcls(defaultUser: String, allowedUsers: String) { + def setViewAcls(defaultUser: String, allowedUsers: Seq[String]) { setViewAcls(Set[String](defaultUser), allowedUsers) } @@ -147,8 +141,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setViewAclsGroups(allowedUserGroups: String) { - viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setViewAclsGroups(allowedUserGroups: Seq[String]) { + viewAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) } @@ -175,8 +169,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) { - modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setModifyAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + modifyAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } @@ -184,8 +178,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setModifyAclsGroups(allowedUserGroups: String) { - modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setModifyAclsGroups(allowedUserGroups: Seq[String]) { + modifyAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) } @@ -212,8 +206,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setAdminAcls(adminUsers: String) { - adminAcls = stringToSet(adminUsers) + def setAdminAcls(adminUsers: Seq[String]) { + adminAcls = adminUsers.toSet logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } @@ -221,8 +215,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setAdminAclsGroups(adminUserGroups: String) { - adminAclsGroups = stringToSet(adminUserGroups) + def setAdminAclsGroups(adminUserGroups: Seq[String]) { + adminAclsGroups = adminUserGroups.toSet logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) } @@ -328,6 +322,7 @@ private[spark] class SecurityManager( .orElse(Option(secretKey)) .orElse(Option(sparkConf.getenv(ENV_AUTH_SECRET))) .orElse(sparkConf.getOption(SPARK_AUTH_SECRET_CONF)) + .orElse(secretKeyFromFile()) .getOrElse { throw new IllegalArgumentException( s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config") @@ -353,20 +348,53 @@ private[spark] class SecurityManager( return } + // TODO: this really should be abstracted somewhere else. val master = sparkConf.get(SparkLauncher.SPARK_MASTER, "") - master match { + val storeInUgi = master match { case "yarn" | "local" | LOCAL_N_REGEX(_) | LOCAL_N_FAILURES_REGEX(_, _) => - // Secret generation allowed here + true + + case k8sRegex() => + // Don't propagate the secret through the user's credentials in kubernetes. That conflicts + // with the way k8s handles propagation of delegation tokens. + false + case _ => require(sparkConf.contains(SPARK_AUTH_SECRET_CONF), s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.") return } - secretKey = Utils.createSecret(sparkConf) - val creds = new Credentials() - creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) - UserGroupInformation.getCurrentUser().addCredentials(creds) + if (sparkConf.get(AUTH_SECRET_FILE_DRIVER).isDefined != + sparkConf.get(AUTH_SECRET_FILE_EXECUTOR).isDefined) { + throw new IllegalArgumentException( + "Invalid secret configuration: Secret files must be specified for both the driver and the" + + " executors, not only one or the other.") + } + + secretKey = secretKeyFromFile().getOrElse(Utils.createSecret(sparkConf)) + + if (storeInUgi) { + val creds = new Credentials() + creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) + UserGroupInformation.getCurrentUser().addCredentials(creds) + } + } + + private def secretKeyFromFile(): Option[String] = { + sparkConf.get(authSecretFileConf).flatMap { secretFilePath => + sparkConf.getOption(SparkLauncher.SPARK_MASTER).map { + case k8sRegex() => + val secretFile = new File(secretFilePath) + require(secretFile.isFile, s"No file found containing the secret key at $secretFilePath.") + val base64Key = Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + require(!base64Key.isEmpty, s"Secret key from file located at $secretFilePath is empty.") + base64Key + case _ => + throw new IllegalArgumentException( + "Secret keys provided via files is only allowed in Kubernetes mode.") + } + } } // Default SecurityManager only has a single secret key, so ignore appId. @@ -376,8 +404,9 @@ private[spark] class SecurityManager( private[spark] object SecurityManager { + val k8sRegex = "k8s.*".r val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key - val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret" + val SPARK_AUTH_SECRET_CONF = AUTH_SECRET.key // This is used to set auth secret to an executor's env variable. It should have the same // value as SPARK_AUTH_SECRET_CONF set in SparkConf val ENV_AUTH_SECRET = "_SPARK_AUTH_SECRET" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 854dfbd7b1cb0..90dfb045b5ea3 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -26,9 +26,10 @@ import scala.concurrent.duration._ import org.apache.avro.{Schema, SchemaNormalization} -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -124,7 +125,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") - set("spark.jars", jars.filter(_ != null).mkString(",")) + set(JARS, jars.filter(_ != null)) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ @@ -202,12 +203,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ def registerKryoClasses(classes: Array[Class[_]]): SparkConf = { val allClassNames = new LinkedHashSet[String]() - allClassNames ++= get("spark.kryo.classesToRegister", "").split(',').map(_.trim) + allClassNames ++= get(KRYO_CLASSES_TO_REGISTER).map(_.trim) .filter(!_.isEmpty) allClassNames ++= classes.map(_.getName) - set("spark.kryo.classesToRegister", allClassNames.mkString(",")) - set("spark.serializer", classOf[KryoSerializer].getName) + set(KRYO_CLASSES_TO_REGISTER, allClassNames.toSeq) + set(SERIALIZER, classOf[KryoSerializer].getName) this } @@ -504,12 +505,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria logWarning(msg) } - val executorOptsKey = "spark.executor.extraJavaOptions" - val executorClasspathKey = "spark.executor.extraClassPath" - val driverOptsKey = "spark.driver.extraJavaOptions" - val driverClassPathKey = "spark.driver.extraClassPath" - val driverLibraryPathKey = "spark.driver.extraLibraryPath" - val sparkExecutorInstances = "spark.executor.instances" + val executorOptsKey = EXECUTOR_JAVA_OPTIONS.key // Used by Yarn in 1.1 and before sys.props.get("spark.driver.libraryPath").foreach { value => @@ -518,7 +514,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria |spark.driver.libraryPath was detected (set to '$value'). |This is deprecated in Spark 1.2+. | - |Please instead use: $driverLibraryPathKey + |Please instead use: ${DRIVER_LIBRARY_PATH.key} """.stripMargin logWarning(warning) } @@ -578,26 +574,26 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria case "yarn-cluster" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "cluster") + set(SUBMIT_DEPLOY_MODE, "cluster") case "yarn-client" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "client") + set(SUBMIT_DEPLOY_MODE, "client") case _ => // Any other unexpected master will be checked when creating scheduler backend. } } - if (contains("spark.submit.deployMode")) { - get("spark.submit.deployMode") match { + if (contains(SUBMIT_DEPLOY_MODE)) { + get(SUBMIT_DEPLOY_MODE) match { case "cluster" | "client" => - case e => throw new SparkException("spark.submit.deployMode can only be \"cluster\" or " + - "\"client\".") + case e => throw new SparkException(s"${SUBMIT_DEPLOY_MODE.key} can only be " + + "\"cluster\" or \"client\".") } } - if (contains("spark.cores.max") && contains("spark.executor.cores")) { - val totalCores = getInt("spark.cores.max", 1) - val executorCores = getInt("spark.executor.cores", 1) + if (contains(CORES_MAX) && contains(EXECUTOR_CORES)) { + val totalCores = getInt(CORES_MAX.key, 1) + val executorCores = get(EXECUTOR_CORES) val leftCores = totalCores % executorCores if (leftCores != 0) { logWarning(s"Total executor cores: ${totalCores} is not " + @@ -606,6 +602,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } + if (contains(EXECUTOR_CORES) && contains("spark.task.cpus")) { + val executorCores = get(EXECUTOR_CORES) + val taskCpus = getInt("spark.task.cpus", 1) + + if (executorCores < taskCpus) { + throw new SparkException(s"${EXECUTOR_CORES.key} must not be less than spark.task.cpus.") + } + } + val encryptionEnabled = get(NETWORK_ENCRYPTION_ENABLED) || get(SASL_ENCRYPTION_ENABLED) require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") @@ -672,7 +677,7 @@ private[spark] object SparkConf extends Logging { * TODO: consolidate it with `ConfigBuilder.withAlternative`. */ private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( - "spark.executor.userClassPathFirst" -> Seq( + EXECUTOR_USER_CLASS_PATH_FIRST.key -> Seq( AlternateConfig("spark.files.userClassPathFirst", "1.3")), "spark.history.fs.update.interval" -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), @@ -695,7 +700,7 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")), "spark.shuffle.file.buffer" -> Seq( AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), - "spark.executor.logs.rolling.maxSize" -> Seq( + EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> Seq( AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), "spark.io.compression.snappy.blockSize" -> Seq( AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), @@ -728,7 +733,13 @@ private[spark] object SparkConf extends Logging { DRIVER_MEMORY_OVERHEAD.key -> Seq( AlternateConfig("spark.yarn.driver.memoryOverhead", "2.3")), EXECUTOR_MEMORY_OVERHEAD.key -> Seq( - AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3")) + AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3")), + KEYTAB.key -> Seq( + AlternateConfig("spark.yarn.keytab", "3.0")), + PRINCIPAL.key -> Seq( + AlternateConfig("spark.yarn.principal", "3.0")), + KERBEROS_RELOGIN_PERIOD.key -> Seq( + AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0")) ) /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e5b1e0ecd1586..98aa636138e30 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -46,6 +46,7 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -53,7 +54,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.local.LocalSchedulerBackend -import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.{AppStatusSource, AppStatusStore} import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump @@ -230,7 +231,7 @@ class SparkContext(config: SparkConf) extends Logging { def jars: Seq[String] = _jars def files: Seq[String] = _files def master: String = _conf.get("spark.master") - def deployMode: String = _conf.getOption("spark.submit.deployMode").getOrElse("client") + def deployMode: String = _conf.get(SUBMIT_DEPLOY_MODE) def appName: String = _conf.get("spark.app.name") private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) @@ -387,9 +388,9 @@ class SparkContext(config: SparkConf) extends Logging { // Set Spark driver host and port system properties. This explicitly sets the configuration // instead of relying on the default value of the config constant. _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_HOST_ADDRESS)) - _conf.setIfMissing("spark.driver.port", "0") + _conf.setIfMissing(DRIVER_PORT, 0) - _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) + _conf.set(EXECUTOR_ID, SparkContext.DRIVER_IDENTIFIER) _jars = Utils.getUserJars(_conf) _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) @@ -417,7 +418,8 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. - _statusStore = AppStatusStore.createLiveStore(conf) + val appStatusSource = AppStatusSource.createSource(conf) + _statusStore = AppStatusStore.createLiveStore(conf, appStatusSource) listenerBus.addToStatusQueue(_statusStore.listener.get) // Create the Spark execution environment (cache, map output tracker, etc) @@ -440,7 +442,7 @@ class SparkContext(config: SparkConf) extends Logging { } _ui = - if (conf.getBoolean("spark.ui.enabled", true)) { + if (conf.get(UI_ENABLED)) { Some(SparkUI.create(Some(this), _statusStore, _conf, _env.securityManager, appName, "", startTime)) } else { @@ -462,7 +464,7 @@ class SparkContext(config: SparkConf) extends Logging { files.foreach(addFile) } - _executorMemory = _conf.getOption("spark.executor.memory") + _executorMemory = _conf.getOption(EXECUTOR_MEMORY.key) .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) .orElse(Option(System.getenv("SPARK_MEM")) .map(warnSparkMem)) @@ -503,7 +505,7 @@ class SparkContext(config: SparkConf) extends Logging { _applicationId = _taskScheduler.applicationId() _applicationAttemptId = taskScheduler.applicationAttemptId() _conf.set("spark.app.id", _applicationId) - if (_conf.getBoolean("spark.ui.reverseProxy", false)) { + if (_conf.get(UI_REVERSE_PROXY)) { System.setProperty("spark.ui.proxyBase", "/proxy/" + _applicationId) } _ui.foreach(_.setAppId(_applicationId)) @@ -563,7 +565,7 @@ class SparkContext(config: SparkConf) extends Logging { _executorAllocationManager.foreach { e => _env.metricsSystem.registerSource(e.executorAllocationManagerSource) } - + appStatusSource.foreach(_env.metricsSystem.registerSource(_)) // Make sure the context is stopped if the user forgets about it. This avoids leaving // unfinished event logs around after the JVM exits cleanly. It doesn't help if the JVM // is killed, though. @@ -2700,8 +2702,8 @@ object SparkContext extends Logging { case SparkMasterRegex.LOCAL_N_REGEX(threads) => convertToInt(threads) case SparkMasterRegex.LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) case "yarn" => - if (conf != null && conf.getOption("spark.submit.deployMode").contains("cluster")) { - conf.getInt("spark.driver.cores", 0) + if (conf != null && conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { + conf.getInt(DRIVER_CORES.key, 0) } else { 0 } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72123f2232532..44074b14eb540 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -163,10 +163,10 @@ object SparkEnv extends Logging { mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { assert(conf.contains(DRIVER_HOST_ADDRESS), s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") - assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") + assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") val bindAddress = conf.get(DRIVER_BIND_ADDRESS) val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) - val port = conf.get("spark.driver.port").toInt + val port = conf.get(DRIVER_PORT) val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(conf)) } else { @@ -232,8 +232,8 @@ object SparkEnv extends Logging { if (isDriver) { assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") } - - val securityManager = new SecurityManager(conf, ioEncryptionKey) + val authSecretFileConf = if (isDriver) AUTH_SECRET_FILE_DRIVER else AUTH_SECRET_FILE_EXECUTOR + val securityManager = new SecurityManager(conf, ioEncryptionKey, authSecretFileConf) if (isDriver) { securityManager.initializeAuth() } @@ -251,7 +251,7 @@ object SparkEnv extends Logging { // Figure out which port RpcEnv actually bound to in case the original port is 0 or occupied. if (isDriver) { - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -274,14 +274,13 @@ object SparkEnv extends Logging { } } - // Create an instance of the class named by the given SparkConf property, or defaultClassName + // Create an instance of the class named by the given SparkConf property // if the property is not set, possibly initializing it with our conf - def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = { - instantiateClass[T](conf.get(propertyName, defaultClassName)) + def instantiateClassFromConf[T](propertyName: ConfigEntry[String]): T = { + instantiateClass[T](conf.get(propertyName)) } - val serializer = instantiateClassFromConf[Serializer]( - "spark.serializer", "org.apache.spark.serializer.JavaSerializer") + val serializer = instantiateClassFromConf[Serializer](SERIALIZER) logDebug(s"Using serializer: ${serializer.getClass}") val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey) @@ -359,7 +358,7 @@ object SparkEnv extends Logging { // We need to set the executor ID before the MetricsSystem is created because sources and // sinks specified in the metrics configuration file will want to incorporate this executor's // ID into the metrics they report. - conf.set("spark.executor.id", executorId) + conf.set(EXECUTOR_ID, executorId) val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) ms.start() ms diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index f73e95eac8f79..5168e9330965d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -27,7 +27,8 @@ import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.internal.config.EXECUTOR_CORES +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -71,11 +72,10 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val conf = SparkEnv.get.conf private val bufferSize = conf.getInt("spark.buffer.size", 65536) - private val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true) + private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. - private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY) - .map(_ / conf.getInt("spark.executor.cores", 1)) + private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY).map(_ / conf.get(EXECUTOR_CORES)) // All the Python functions should have the same exec, version and envvars. protected val envVars = funcs.head.funcs.head.envVars @@ -496,7 +496,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( extends Thread(s"Worker Monitor for $pythonExec") { /** How long to wait before killing the python worker if a task cannot be interrupted. */ - private val taskKillTimeout = env.conf.getTimeAsMs("spark.python.task.killTimeout", "2s") + private val taskKillTimeout = env.conf.get(PYTHON_TASK_KILL_TIMEOUT) setDaemon(true) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 6afa37aa36fd3..829a4b18f0209 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -27,6 +27,7 @@ import scala.collection.mutable import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util.{RedirectThread, Utils} @@ -39,8 +40,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // pyspark/daemon.py (by default) and tell it to fork new workers for our tasks. This daemon // currently only works on UNIX-based systems now because it uses signals for child management, // so we can also fall back to launching workers, pyspark/worker.py (by default) directly. - val useDaemon = { - val useDaemonEnabled = SparkEnv.get.conf.getBoolean("spark.python.use.daemon", true) + private val useDaemon = { + val useDaemonEnabled = SparkEnv.get.conf.get(PYTHON_USE_DAEMON) // This flag is ignored on Windows as it's unable to fork. !System.getProperty("os.name").startsWith("Windows") && useDaemonEnabled @@ -51,22 +52,24 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // as expert-only option, and shouldn't be used before knowing what it means exactly. // This configuration indicates the module to run the daemon to execute its Python workers. - val daemonModule = SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value => - logInfo( - s"Python daemon module in PySpark is set to [$value] in 'spark.python.daemon.module', " + - "using this to start the daemon up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is enabled and the platform is not Windows.") - value - }.getOrElse("pyspark.daemon") + private val daemonModule = + SparkEnv.get.conf.get(PYTHON_DAEMON_MODULE).map { value => + logInfo( + s"Python daemon module in PySpark is set to [$value] in '${PYTHON_DAEMON_MODULE.key}', " + + "using this to start the daemon up. Note that this configuration only has an effect when " + + s"'${PYTHON_USE_DAEMON.key}' is enabled and the platform is not Windows.") + value + }.getOrElse("pyspark.daemon") // This configuration indicates the module to run each Python worker. - val workerModule = SparkEnv.get.conf.getOption("spark.python.worker.module").map { value => - logInfo( - s"Python worker module in PySpark is set to [$value] in 'spark.python.worker.module', " + - "using this to start the worker up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is disabled or the platform is Windows.") - value - }.getOrElse("pyspark.worker") + private val workerModule = + SparkEnv.get.conf.get(PYTHON_WORKER_MODULE).map { value => + logInfo( + s"Python worker module in PySpark is set to [$value] in '${PYTHON_WORKER_MODULE.key}', " + + "using this to start the worker up. Note that this configuration only has an effect when " + + s"'${PYTHON_USE_DAEMON.key}' is disabled or the platform is Windows.") + value + }.getOrElse("pyspark.worker") private val authHelper = new SocketAuthHelper(SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala index 7ce2581555014..c7c4384bda002 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -32,6 +32,7 @@ import io.netty.handler.timeout.ReadTimeoutHandler import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.Utils @@ -49,10 +50,8 @@ private[spark] class RBackend { def init(): (Int, RAuthHelper) = { val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) - bossGroup = new NioEventLoopGroup( - conf.getInt("spark.r.numRBackendThreads", SparkRDefaults.DEFAULT_NUM_RBACKEND_THREADS)) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) + bossGroup = new NioEventLoopGroup(conf.get(R_NUM_BACKEND_THREADS)) val workerGroup = bossGroup val handler = new RBackendHandler(this) val authHelper = new RAuthHelper(conf) @@ -128,8 +127,7 @@ private[spark] object RBackend extends Logging { // Connection timeout is set by socket client. To make it configurable we will pass the // timeout value to client inside the temp file val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) // tell the R process via temporary file val path = args(0) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 18fc595301f46..7b74efa41044f 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -29,6 +29,7 @@ import io.netty.handler.timeout.ReadTimeoutException import org.apache.spark.SparkConf import org.apache.spark.api.r.SerDe._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -98,10 +99,8 @@ private[r] class RBackendHandler(server: RBackend) } } val conf = new SparkConf() - val heartBeatInterval = conf.getInt( - "spark.r.heartBeatInterval", SparkRDefaults.DEFAULT_HEARTBEAT_INTERVAL) - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val heartBeatInterval = conf.get(R_HEARTBEAT_INTERVAL) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) val interval = Math.min(heartBeatInterval, backendConnectionTimeout - 1) execService.scheduleAtFixedRate(pingRunner, interval, interval, TimeUnit.SECONDS) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index e7fdc3963945a..3fdea04cdf7a7 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -27,6 +27,7 @@ import scala.util.Try import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.util.Utils /** @@ -340,11 +341,10 @@ private[r] object RRunner { // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", // but kept here for backward compatibility. val sparkConf = SparkEnv.get.conf - var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") - rCommand = sparkConf.get("spark.r.command", rCommand) + var rCommand = sparkConf.get(SPARKR_COMMAND) + rCommand = sparkConf.get(R_COMMAND).orElse(Some(rCommand)).get - val rConnectionTimeout = sparkConf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val rConnectionTimeout = sparkConf.get(R_BACKEND_CONNECTION_TIMEOUT) val rOptions = "--vanilla" val rLibDir = RUtils.sparkRPackagePath(isDriver = false) val rExecScript = rLibDir(0) + "/SparkR/worker/" + script diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala index 9bf35af1da925..6832223a5daf3 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -23,6 +23,7 @@ import java.util.Arrays import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.api.python.PythonUtils +import org.apache.spark.internal.config._ private[spark] object RUtils { // Local path where R binary packages built from R source code contained in the spark @@ -63,7 +64,7 @@ private[spark] object RUtils { (sys.props("spark.master"), sys.props("spark.submit.deployMode")) } else { val sparkConf = SparkEnv.get.conf - (sparkConf.get("spark.master"), sparkConf.get("spark.submit.deployMode", "client")) + (sparkConf.get("spark.master"), sparkConf.get(SUBMIT_DEPLOY_MODE)) } val isYarnCluster = master != null && master.contains("yarn") && deployMode == "cluster" diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 708910bed86cc..e28f92332075d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -27,7 +27,7 @@ import org.apache.log4j.Logger import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils} @@ -72,18 +72,18 @@ private class ClientEndpoint( // people call `addJar` assuming the jar is in the same directory. val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" - val classPathConf = "spark.driver.extraClassPath" - val classPathEntries = getProperty(classPathConf, conf).toSeq.flatMap { cp => + val classPathConf = config.DRIVER_CLASS_PATH.key + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val libraryPathConf = "spark.driver.extraLibraryPath" - val libraryPathEntries = getProperty(libraryPathConf, conf).toSeq.flatMap { cp => + val libraryPathConf = config.DRIVER_LIBRARY_PATH.key + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val extraJavaOptsConf = "spark.driver.extraJavaOptions" - val extraJavaOpts = getProperty(extraJavaOptsConf, conf) + val extraJavaOptsConf = config.DRIVER_JAVA_OPTIONS.key + val extraJavaOpts = sys.props.get(extraJavaOptsConf) .map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index c6307da61c7eb..a66243012041c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -34,7 +34,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -60,7 +60,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private object FaultToleranceTest extends App with Logging { private val conf = new SparkConf() - private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val zkDir = conf.get(config.Deploy.ZOOKEEPER_DIRECTORY).getOrElse("/spark") private val masters = ListBuffer[TestMasterInfo]() private val workers = ListBuffer[TestWorkerInfo]() @@ -77,7 +77,7 @@ private object FaultToleranceTest extends App with Logging { private val containerSparkHome = "/opt/spark" private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) - System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip + System.setProperty(config.DRIVER_HOST_ADDRESS.key, "172.17.42.1") // default docker host ip private def afterEach() { if (sc != null) { @@ -87,8 +87,8 @@ private object FaultToleranceTest extends App with Logging { terminateCluster() // Clear ZK directories in between tests (for speed purposes) - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/spark_leader") - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/master_status") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/spark_leader") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/master_status") } test("sanity-basic") { @@ -216,7 +216,7 @@ private object FaultToleranceTest extends App with Logging { if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. - System.setProperty("spark.driver.port", "0") + System.setProperty(config.DRIVER_PORT.key, "0") sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index be293f88a9d4a..c1866b4c3606e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -51,8 +51,8 @@ class LocalSparkCluster( // Disable REST server on Master in this mode unless otherwise specified val _conf = conf.clone() - .setIfMissing("spark.master.rest.enabled", "false") - .set(config.SHUFFLE_SERVICE_ENABLED.key, "false") + .setIfMissing(config.MASTER_REST_SERVER_ENABLED, false) + .set(config.SHUFFLE_SERVICE_ENABLED, false) /* Start the Master */ val (rpcEnv, webUiPort, _) = Master.startRpcEnvAndEndpoint(localHostname, 0, 0, _conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index e86b362639e57..6284e6a6448f8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -25,7 +25,8 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkUserAppException} -import org.apache.spark.api.r.{RBackend, RUtils, SparkRDefaults} +import org.apache.spark.api.r.{RBackend, RUtils} +import org.apache.spark.internal.config.R._ import org.apache.spark.util.RedirectThread /** @@ -43,8 +44,8 @@ object RRunner { val rCommand = { // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", // but kept here for backward compatibility. - var cmd = sys.props.getOrElse("spark.sparkr.r.command", "Rscript") - cmd = sys.props.getOrElse("spark.r.command", cmd) + var cmd = sys.props.getOrElse(SPARKR_COMMAND.key, SPARKR_COMMAND.defaultValue.get) + cmd = sys.props.getOrElse(R_COMMAND.key, cmd) if (sys.props.getOrElse("spark.submit.deployMode", "client") == "client") { cmd = sys.props.getOrElse("spark.r.driver.command", cmd) } @@ -53,7 +54,7 @@ object RRunner { // Connection timeout set by R process on its connection to RBackend in seconds. val backendConnectionTimeout = sys.props.getOrElse( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT.toString) + R_BACKEND_CONNECTION_TIMEOUT.key, R_BACKEND_CONNECTION_TIMEOUT.defaultValue.get.toString) // Check if the file path exists. // If not, change directory to current working directory for YARN cluster mode diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 8247110940dbc..8118c01eb712f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -25,6 +25,7 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL private[spark] object SparkCuratorUtil extends Logging { @@ -35,7 +36,7 @@ private[spark] object SparkCuratorUtil extends Logging { def newClient( conf: SparkConf, - zkUrlConf: String = "spark.deploy.zookeeper.url"): CuratorFramework = { + zkUrlConf: String = ZOOKEEPER_URL.key): CuratorFramework = { val ZK_URL = conf.get(zkUrlConf) val zk = CuratorFrameworkFactory.newClient(ZK_URL, ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 4cc0063d010ef..7b0c2dfb4c524 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -412,20 +412,6 @@ object SparkHadoopUtil { def get: SparkHadoopUtil = instance - /** - * Given an expiration date for the current set of credentials, calculate the time when new - * credentials should be created. - * - * @param expirationDate Drop-dead expiration date - * @param conf Spark configuration - * @return Timestamp when new credentials should be created. - */ - private[spark] def nextCredentialRenewalTime(expirationDate: Long, conf: SparkConf): Long = { - val ct = System.currentTimeMillis - val ratio = conf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) - (ct + (ratio * (expirationDate - ct))).toLong - } - /** * Returns a Configuration object with Spark configuration applied on top. Unlike * the instance method, this will always return a Configuration instance, and not a diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 385e1c68e9621..38137f6f156ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -22,9 +22,10 @@ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowab import java.net.URL import java.security.PrivilegedExceptionAction import java.text.ParseException -import java.util.UUID +import java.util.{ServiceLoader, UUID} import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import scala.util.{Properties, Try} @@ -50,6 +51,7 @@ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util._ @@ -95,20 +97,35 @@ private[spark] class SparkSubmit extends Logging { } /** - * Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only. + * Kill an existing submission. */ private def kill(args: SparkSubmitArguments): Unit = { - new RestSubmissionClient(args.master) - .killSubmission(args.submissionToKill) + if (RestSubmissionClient.supportsRestClient(args.master)) { + new RestSubmissionClient(args.master) + .killSubmission(args.submissionToKill) + } else { + val sparkConf = args.toSparkConf() + sparkConf.set("spark.master", args.master) + SparkSubmitUtils + .getSubmitOperations(args.master) + .kill(args.submissionToKill, sparkConf) + } } /** - * Request the status of an existing submission using the REST protocol. - * Standalone and Mesos cluster mode only. + * Request the status of an existing submission. */ private def requestStatus(args: SparkSubmitArguments): Unit = { - new RestSubmissionClient(args.master) - .requestSubmissionStatus(args.submissionToRequestStatusFor) + if (RestSubmissionClient.supportsRestClient(args.master)) { + new RestSubmissionClient(args.master) + .requestSubmissionStatus(args.submissionToRequestStatusFor) + } else { + val sparkConf = args.toSparkConf() + sparkConf.set("spark.master", args.master) + SparkSubmitUtils + .getSubmitOperations(args.master) + .printSubmissionStatus(args.submissionToRequestStatusFor, sparkConf) + } } /** Print version information to the log. */ @@ -330,12 +347,13 @@ private[spark] class SparkSubmit extends Logging { } } - args.sparkProperties.foreach { case (k, v) => sparkConf.set(k, v) } + // update spark config from args + args.toSparkConf(Option(sparkConf)) val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf)) val targetDir = Utils.createTempDir() // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient) { + if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) { if (args.principal != null) { if (args.keytab != null) { require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") @@ -435,7 +453,7 @@ private[spark] class SparkSubmit extends Logging { } if (localPyFiles != null) { - sparkConf.set("spark.submit.pyFiles", localPyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, localPyFiles.split(",").toSeq) } // In YARN mode for an R app, add the SparkR package archive and the R package @@ -513,13 +531,18 @@ private[spark] class SparkSubmit extends Logging { OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = "spark.app.name"), OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, confKey = "spark.jars.ivy"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraClassPath"), + confKey = DRIVER_CLASS_PATH.key), OptionAssigner(args.driverExtraJavaOptions, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraJavaOptions"), + confKey = DRIVER_JAVA_OPTIONS.key), OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraLibraryPath"), + confKey = DRIVER_LIBRARY_PATH.key), + OptionAssigner(args.principal, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, + confKey = PRINCIPAL.key), + OptionAssigner(args.keytab, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, + confKey = KEYTAB.key), + OptionAssigner(args.pyFiles, ALL_CLUSTER_MGRS, CLUSTER, confKey = SUBMIT_PYTHON_FILES.key), // Propagate attributes for dependency resolution at the driver side OptionAssigner(args.packages, STANDALONE | MESOS, CLUSTER, confKey = "spark.jars.packages"), @@ -531,8 +554,6 @@ private[spark] class SparkSubmit extends Logging { // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.queue"), - OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, - confKey = "spark.executor.instances"), OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.pyFiles"), OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.jars"), OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.files"), @@ -541,23 +562,25 @@ private[spark] class SparkSubmit extends Logging { OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.keytab"), // Other options + OptionAssigner(args.numExecutors, YARN | KUBERNETES, ALL_DEPLOY_MODES, + confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.cores"), + confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.memory"), + confKey = EXECUTOR_MEMORY.key), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.cores.max"), + confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, confKey = "spark.jars"), OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.cores"), + confKey = DRIVER_CORES.key), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, - confKey = "spark.driver.supervise"), + confKey = DRIVER_SUPERVISE.key), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = "spark.jars.ivy"), // An internal option used only for spark-shell to add user jars to repl's classloader, @@ -610,11 +633,11 @@ private[spark] class SparkSubmit extends Logging { // For YARN cluster mode, the jar is already distributed on each node as "app.jar" // For python and R files, the primary resource is already distributed as a regular file if (!isYarnCluster && !args.isPython && !args.isR) { - var jars = sparkConf.getOption("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) + var jars = sparkConf.get(JARS) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } - sparkConf.set("spark.jars", jars.mkString(",")) + sparkConf.set(JARS, jars) } // In standalone cluster mode, use the REST client to submit the application (Spark 1.3+). @@ -644,7 +667,8 @@ private[spark] class SparkSubmit extends Logging { } } - if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { + if ((clusterManager == MESOS || clusterManager == KUBERNETES) + && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } @@ -676,7 +700,7 @@ private[spark] class SparkSubmit extends Logging { // Second argument is main class childArgs += (args.primaryResource, "") if (args.pyFiles != null) { - sparkConf.set("spark.submit.pyFiles", args.pyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, args.pyFiles.split(",").toSeq) } } else if (args.isR) { // Second argument is main class @@ -695,9 +719,6 @@ private[spark] class SparkSubmit extends Logging { if (args.isPython) { childArgs ++= Array("--primary-py-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") - if (args.pyFiles != null) { - childArgs ++= Array("--other-py-files", args.pyFiles) - } } else if (args.isR) { childArgs ++= Array("--primary-r-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.RRunner") @@ -723,7 +744,7 @@ private[spark] class SparkSubmit extends Logging { // Ignore invalid spark.driver.host in cluster modes. if (deployMode == CLUSTER) { - sparkConf.remove("spark.driver.host") + sparkConf.remove(DRIVER_HOST_ADDRESS) } // Resolve paths in certain spark properties @@ -743,25 +764,24 @@ private[spark] class SparkSubmit extends Logging { // Resolve and format python file paths properly before adding them to the PYTHONPATH. // The resolving part is redundant in the case of --py-files, but necessary if the user // explicitly sets `spark.submit.pyFiles` in his/her default properties file. - sparkConf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val resolvedPyFiles = Utils.resolveURIs(pyFiles) - val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { - PythonRunner.formatPaths(resolvedPyFiles).mkString(",") - } else { - // Ignoring formatting python path in yarn and mesos cluster mode, these two modes - // support dealing with remote python files, they could distribute and add python files - // locally. - resolvedPyFiles - } - sparkConf.set("spark.submit.pyFiles", formattedPyFiles) + val pyFiles = sparkConf.get(SUBMIT_PYTHON_FILES) + val resolvedPyFiles = Utils.resolveURIs(pyFiles.mkString(",")) + val formattedPyFiles = if (deployMode != CLUSTER) { + PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + } else { + // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // support dealing with remote python files, they could distribute and add python files + // locally. + resolvedPyFiles } + sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toSeq) (childArgs, childClasspath, sparkConf, childMainClass) } // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with - // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we - // must trick it into thinking we're YARN. + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos or Kubernetes + // mode, we must trick it into thinking we're YARN. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" @@ -1338,6 +1358,23 @@ private[spark] object SparkSubmitUtils { } } + private[deploy] def getSubmitOperations(master: String): SparkSubmitOperation = { + val loader = Utils.getContextOrSparkClassLoader + val serviceLoaders = + ServiceLoader.load(classOf[SparkSubmitOperation], loader) + .asScala + .filter(_.supports(master)) + + serviceLoaders.size match { + case x if x > 1 => + throw new SparkException(s"Multiple($x) external SparkSubmitOperations " + + s"clients registered for master url ${master}.") + case 1 => serviceLoaders.headOption.get + case _ => + throw new IllegalArgumentException(s"No external SparkSubmitOperations " + + s"clients found for master url: '$master'") + } + } } /** @@ -1350,3 +1387,12 @@ private case class OptionAssigner( deployMode: Int, clOption: String = null, confKey: String = null) + +private[spark] trait SparkSubmitOperation { + + def kill(submissionId: String, conf: SparkConf): Unit + + def printSubmissionStatus(submissionId: String, conf: SparkConf): Unit + + def supports(master: String): Boolean +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 0998757715457..70121056c5e2b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -29,9 +29,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.io.Source import scala.util.Try -import org.apache.spark.{SparkException, SparkUserAppException} +import org.apache.spark.{SparkConf, SparkException, SparkUserAppException} import org.apache.spark.deploy.SparkSubmitAction._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkSubmitArgumentsParser import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.Utils @@ -155,31 +155,31 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(env.get("MASTER")) .orNull driverExtraClassPath = Option(driverExtraClassPath) - .orElse(sparkProperties.get("spark.driver.extraClassPath")) + .orElse(sparkProperties.get(config.DRIVER_CLASS_PATH.key)) .orNull driverExtraJavaOptions = Option(driverExtraJavaOptions) - .orElse(sparkProperties.get("spark.driver.extraJavaOptions")) + .orElse(sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key)) .orNull driverExtraLibraryPath = Option(driverExtraLibraryPath) - .orElse(sparkProperties.get("spark.driver.extraLibraryPath")) + .orElse(sparkProperties.get(config.DRIVER_LIBRARY_PATH.key)) .orNull driverMemory = Option(driverMemory) - .orElse(sparkProperties.get("spark.driver.memory")) + .orElse(sparkProperties.get(config.DRIVER_MEMORY.key)) .orElse(env.get("SPARK_DRIVER_MEMORY")) .orNull driverCores = Option(driverCores) - .orElse(sparkProperties.get("spark.driver.cores")) + .orElse(sparkProperties.get(config.DRIVER_CORES.key)) .orNull executorMemory = Option(executorMemory) - .orElse(sparkProperties.get("spark.executor.memory")) + .orElse(sparkProperties.get(config.EXECUTOR_MEMORY.key)) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) .orNull executorCores = Option(executorCores) - .orElse(sparkProperties.get("spark.executor.cores")) + .orElse(sparkProperties.get(config.EXECUTOR_CORES.key)) .orElse(env.get("SPARK_EXECUTOR_CORES")) .orNull totalExecutorCores = Option(totalExecutorCores) - .orElse(sparkProperties.get("spark.cores.max")) + .orElse(sparkProperties.get(config.CORES_MAX.key)) .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull @@ -197,7 +197,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(env.get("DEPLOY_MODE")) .orNull numExecutors = Option(numExecutors) - .getOrElse(sparkProperties.get("spark.executor.instances").orNull) + .getOrElse(sparkProperties.get(config.EXECUTOR_INSTANCES.key).orNull) queue = Option(queue).orElse(sparkProperties.get("spark.yarn.queue")).orNull keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull @@ -298,19 +298,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { - error("Killing submissions is only supported in standalone or Mesos mode!") - } if (submissionToKill == null) { error("Please specify a submission to kill.") } } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { - error( - "Requesting submission statuses is only supported in standalone or Mesos mode!") - } if (submissionToRequestStatusFor == null) { error("Please specify a submission to request status for.") } @@ -567,6 +560,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | | Spark standalone or Mesos with cluster deploy mode only: | --supervise If given, restarts the driver on failure. + | + | Spark standalone, Mesos or K8s with cluster deploy mode only: | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | @@ -577,20 +572,18 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, | or all available cores on the worker in standalone mode) | - | YARN-only: - | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). + | Spark on YARN and Kubernetes only: | --num-executors NUM Number of executors to launch (Default: 2). | If dynamic allocation is enabled, the initial number of | executors will be at least NUM. + | --principal PRINCIPAL Principal to be used to login to KDC. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. + | + | Spark on YARN only: + | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. - | --principal PRINCIPAL Principal to be used to login to KDC, while running on - | secure HDFS. - | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. This keytab will be copied to - | the node running the Application Master via the Secure - | Distributed Cache, for renewing the login tickets and the - | delegation tokens periodically. """.stripMargin ) @@ -656,4 +649,10 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S private def error(msg: String): Unit = throw new SparkException(msg) + private[deploy] def toSparkConf(sparkConf: Option[SparkConf] = None): SparkConf = { + // either use an existing config or create a new empty one + sparkProperties.foldLeft(sparkConf.getOrElse(new SparkConf())) { + case (conf, (k, v)) => conf.set(k, v) + } + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 38ed5e0bc169e..6ab6c84a3c580 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -42,13 +42,16 @@ import org.fusesource.leveldbjni.internal.NativeDB import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Status._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.status._ import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} -import org.apache.spark.status.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -86,7 +89,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) this(conf, new SystemClock()) } - import config._ import FsHistoryProvider._ // Interval between safemode checks. @@ -105,12 +107,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val logDir = conf.get(EVENT_LOG_DIR) - private val HISTORY_UI_ACLS_ENABLE = conf.getBoolean("spark.history.ui.acls.enable", false) - private val HISTORY_UI_ADMIN_ACLS = conf.get("spark.history.ui.admin.acls", "") - private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get("spark.history.ui.admin.acls.groups", "") - logInfo(s"History server ui acls " + (if (HISTORY_UI_ACLS_ENABLE) "enabled" else "disabled") + - "; users with admin permissions: " + HISTORY_UI_ADMIN_ACLS.toString + - "; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString) + private val historyUiAclsEnable = conf.get(History.HISTORY_SERVER_UI_ACLS_ENABLE) + private val historyUiAdminAcls = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS) + private val historyUiAdminAclsGroups = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS) + logInfo(s"History server ui acls " + (if (historyUiAclsEnable) "enabled" else "disabled") + + "; users with admin permissions: " + historyUiAdminAcls.mkString(",") + + "; groups with admin permissions" + historyUiAdminAclsGroups.mkString(",")) private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) // Visible for testing @@ -307,6 +309,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() + /** + * Split a comma separated String, filter out any empty items, and return a Sequence of strings + */ + private def stringToSeq(list: String): Seq[String] = { + list.split(',').map(_.trim).filter(!_.isEmpty) + } + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { val app = try { load(appId) @@ -323,13 +332,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val conf = this.conf.clone() val secManager = new SecurityManager(conf) - secManager.setAcls(HISTORY_UI_ACLS_ENABLE) + secManager.setAcls(historyUiAclsEnable) // make sure to set admin acls before view acls so they are properly picked up - secManager.setAdminAcls(HISTORY_UI_ADMIN_ACLS + "," + attempt.adminAcls.getOrElse("")) - secManager.setViewAcls(attempt.info.sparkUser, attempt.viewAcls.getOrElse("")) - secManager.setAdminAclsGroups(HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - attempt.adminAclsGroups.getOrElse("")) - secManager.setViewAclsGroups(attempt.viewAclsGroups.getOrElse("")) + secManager.setAdminAcls(historyUiAdminAcls ++ stringToSeq(attempt.adminAcls.getOrElse(""))) + secManager.setViewAcls(attempt.info.sparkUser, stringToSeq(attempt.viewAcls.getOrElse(""))) + secManager.setAdminAclsGroups(historyUiAdminAclsGroups ++ + stringToSeq(attempt.adminAclsGroups.getOrElse(""))) + secManager.setViewAclsGroups(stringToSeq(attempt.viewAclsGroups.getOrElse(""))) val kvstore = try { diskManager match { @@ -1119,11 +1128,16 @@ private[history] class AppListingListener( // Only parse the first env update, since any future changes don't have any effect on // the ACLs set for the UI. if (!gotEnvUpdate) { + def emptyStringToNone(strOption: Option[String]): Option[String] = strOption match { + case Some("") => None + case _ => strOption + } + val allProperties = event.environmentDetails("Spark Properties").toMap - attempt.viewAcls = allProperties.get("spark.ui.view.acls") - attempt.adminAcls = allProperties.get("spark.admin.acls") - attempt.viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") - attempt.adminAclsGroups = allProperties.get("spark.admin.acls.groups") + attempt.viewAcls = emptyStringToNone(allProperties.get(UI_VIEW_ACLS.key)) + attempt.adminAcls = emptyStringToNone(allProperties.get(ADMIN_ACLS.key)) + attempt.viewAclsGroups = emptyStringToNone(allProperties.get(UI_VIEW_ACLS_GROUPS.key)) + attempt.adminAclsGroups = emptyStringToNone(allProperties.get(ADMIN_ACLS_GROUPS.key)) gotEnvUpdate = true checkProgress() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 56f3f59504a7d..7b9814d70b85f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -28,9 +28,10 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.config.HISTORY_SERVER_UI_PORT import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History.HISTORY_SERVER_UI_PORT +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -304,11 +305,10 @@ object HistoryServer extends Logging { config.set(SecurityManager.SPARK_AUTH_CONF, "false") } - if (config.getBoolean("spark.acls.enable", config.getBoolean("spark.ui.acls.enable", false))) { - logInfo("Either spark.acls.enable or spark.ui.acls.enable is configured, clearing it and " + - "only using spark.history.ui.acl.enable") - config.set("spark.acls.enable", "false") - config.set("spark.ui.acls.enable", "false") + if (config.get(ACLS_ENABLE)) { + logInfo(s"${ACLS_ENABLE.key} is configured, " + + s"clearing it and only using ${History.HISTORY_SERVER_UI_ACLS_ENABLE.key}") + config.set(ACLS_ENABLE, false) } new SecurityManager(config) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index ad0dd23cb59c8..0a1f33395ad62 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History._ import org.apache.spark.status.KVUtils._ import org.apache.spark.util.{Clock, Utils} import org.apache.spark.util.kvstore.KVStore @@ -50,8 +51,6 @@ private class HistoryServerDiskManager( listing: KVStore, clock: Clock) extends Logging { - import config._ - private val appStoreDir = new File(path, "apps") if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) { throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index e1184248af460..b26da8a467fd5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -33,6 +33,10 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.serializer.{JavaSerializer, Serializer} @@ -54,12 +58,12 @@ private[deploy] class Master( // For application IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) - private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000 - private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") - private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10) + private val workerTimeoutMs = conf.get(WORKER_TIMEOUT) * 1000 + private val retainedApplications = conf.get(RETAINED_APPLICATIONS) + private val retainedDrivers = conf.get(RETAINED_DRIVERS) + private val reaperIterations = conf.get(REAPER_ITERATIONS) + private val recoveryMode = conf.get(RECOVERY_MODE) + private val maxExecutorRetries = conf.get(MAX_EXECUTOR_RETRIES) val workers = new HashSet[WorkerInfo] val idToApp = new HashMap[String, ApplicationInfo] @@ -111,17 +115,17 @@ private[deploy] class Master( // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.get(SPREAD_OUT_APPS) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + private val defaultCores = conf.get(DEFAULT_CORES) + val reverseProxy = conf.get(UI_REVERSE_PROXY) if (defaultCores < 1) { - throw new SparkException("spark.deploy.defaultCores must be positive") + throw new SparkException(s"${DEFAULT_CORES.key} must be positive") } // Alternative application submission gateway that is stable across Spark versions - private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", false) + private val restServerEnabled = conf.get(MASTER_REST_SERVER_ENABLED) private var restServer: Option[StandaloneRestServer] = None private var restServerBoundPort: Option[Int] = None @@ -140,7 +144,7 @@ private[deploy] class Master( webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort if (reverseProxy) { - masterWebUiUrl = conf.get("spark.ui.reverseProxyUrl", masterWebUiUrl) + masterWebUiUrl = conf.get(UI_REVERSE_PROXY_URL).orElse(Some(masterWebUiUrl)).get webUi.addProxy() logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " + s"Applications UIs are available at $masterWebUiUrl") @@ -149,10 +153,10 @@ private[deploy] class Master( override def run(): Unit = Utils.tryLogNonFatalError { self.send(CheckForWorkerTimeOut) } - }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + }, 0, workerTimeoutMs, TimeUnit.MILLISECONDS) if (restServerEnabled) { - val port = conf.getInt("spark.master.rest.port", 6066) + val port = conf.get(MASTER_REST_SERVER_PORT) restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) } restServerBoundPort = restServer.map(_.start()) @@ -166,7 +170,7 @@ private[deploy] class Master( applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) val serializer = new JavaSerializer(conf) - val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { + val (persistenceEngine_, leaderElectionAgent_) = recoveryMode match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") val zkFactory = @@ -177,7 +181,7 @@ private[deploy] class Master( new FileSystemRecoveryModeFactory(conf, serializer) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => - val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory")) + val clazz = Utils.classForName(conf.get(RECOVERY_MODE_FACTORY)) val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serializer]) .newInstance(conf, serializer) .asInstanceOf[StandaloneRecoveryModeFactory] @@ -231,7 +235,7 @@ private[deploy] class Master( override def run(): Unit = Utils.tryLogNonFatalError { self.send(CompleteRecovery) } - }, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + }, workerTimeoutMs, TimeUnit.MILLISECONDS) } case CompleteRecovery => completeRecovery() @@ -309,8 +313,8 @@ private[deploy] class Master( // Important note: this code path is not exercised by tests, so be very careful when // changing this `if` condition. if (!normalExit - && appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES - && MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path + && appInfo.incrementRetryCount() >= maxExecutorRetries + && maxExecutorRetries >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values if (!execs.exists(_.state == ExecutorState.RUNNING)) { logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + @@ -868,8 +872,8 @@ private[deploy] class Master( endpointToApp -= app.driver addressToApp -= app.driver.address - if (completedApps.size >= RETAINED_APPLICATIONS) { - val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) + if (completedApps.size >= retainedApplications) { + val toRemove = math.max(retainedApplications / 10, 1) completedApps.take(toRemove).foreach { a => applicationMetricsSystem.removeSource(a.appSource) } @@ -987,14 +991,14 @@ private[deploy] class Master( private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() - val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT_MS).toArray + val toRemove = workers.filter(_.lastHeartbeat < currentTime - workerTimeoutMs).toArray for (worker <- toRemove) { if (worker.state != WorkerState.DEAD) { logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT_MS / 1000)) - removeWorker(worker, s"Not receiving heartbeat for ${WORKER_TIMEOUT_MS / 1000} seconds") + worker.id, workerTimeoutMs / 1000)) + removeWorker(worker, s"Not receiving heartbeat for ${workerTimeoutMs / 1000} seconds") } else { - if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT_MS)) { + if (worker.lastHeartbeat < currentTime - ((reaperIterations + 1) * workerTimeoutMs)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it } } @@ -1029,8 +1033,8 @@ private[deploy] class Master( case Some(driver) => logInfo(s"Removing driver: $driverId") drivers -= driver - if (completedDrivers.size >= RETAINED_DRIVERS) { - val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + if (completedDrivers.size >= retainedDrivers) { + val toRemove = math.max(retainedDrivers / 10, 1) completedDrivers.trimStart(toRemove) } completedDrivers += driver diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 615d2533cf085..cd31bbdcfab59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -21,6 +21,7 @@ import scala.annotation.tailrec import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.MASTER_UI_PORT import org.apache.spark.util.{IntParam, Utils} /** @@ -53,8 +54,8 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.master.ui.port")) { - webUiPort = conf.get("spark.master.ui.port").toInt + if (conf.contains(MASTER_UI_PORT.key)) { + webUiPort = conf.get(MASTER_UI_PORT) } @tailrec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index ffdd635be4f5f..470798793cecb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.RECOVERY_DIRECTORY import org.apache.spark.serializer.Serializer /** @@ -52,11 +53,11 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serializer) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { - val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + val recoveryDir = conf.get(RECOVERY_DIRECTORY) def createPersistenceEngine(): PersistenceEngine = { - logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) + logInfo("Persisting recovery state to directory: " + recoveryDir) + new FileSystemPersistenceEngine(recoveryDir, serializer) } def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 1e8dabfbe6b0c..47f309144bdc0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -23,11 +23,12 @@ import org.apache.curator.framework.recipes.leader.{LeaderLatch, LeaderLatchList import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_DIRECTORY private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/leader_election" private var zk: CuratorFramework = _ private var leaderLatch: LeaderLatch = _ @@ -38,7 +39,7 @@ private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderEle private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) - leaderLatch = new LeaderLatch(zk, WORKING_DIR) + leaderLatch = new LeaderLatch(zk, workingDir) leaderLatch.addListener(this) leaderLatch.start() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index af850e4871e57..73dd0de017960 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -28,6 +28,7 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.serializer.Serializer @@ -35,22 +36,22 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer extends PersistenceEngine with Logging { - private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/master_status" private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) override def persist(name: String, obj: Object): Unit = { - serializeIntoFile(WORKING_DIR + "/" + name, obj) + serializeIntoFile(workingDir + "/" + name, obj) } override def unpersist(name: String): Unit = { - zk.delete().forPath(WORKING_DIR + "/" + name) + zk.delete().forPath(workingDir + "/" + name) } override def read[T: ClassTag](prefix: String): Seq[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala + zk.getChildren.forPath(workingDir).asScala .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]) } @@ -66,13 +67,13 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer } private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { - val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val fileData = zk.getData().forPath(workingDir + "/" + filename) try { Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData))) } catch { case e: Exception => logWarning("Exception while reading persisted file, deleting", e) - zk.delete().forPath(WORKING_DIR + "/" + filename) + zk.delete().forPath(workingDir + "/" + filename) None } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index e87b2240564bd..be402ae247511 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -34,7 +35,7 @@ class MasterWebUI( requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self - val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) + val killEnabled = master.conf.get(UI_KILL_ENABLED) initialize() diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index afa413fe165df..1648ba516d9b6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -61,8 +61,6 @@ import org.apache.spark.util.Utils private[spark] class RestSubmissionClient(master: String) extends Logging { import RestSubmissionClient._ - private val supportedMasterPrefixes = Seq("spark://", "mesos://") - private val masters: Array[String] = if (master.startsWith("spark://")) { Utils.parseStandaloneMasterUrls(master) } else { @@ -409,6 +407,8 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { private[spark] object RestSubmissionClient { + val supportedMasterPrefixes = Seq("spark://", "mesos://") + // SPARK_HOME and SPARK_CONF_DIR are filtered out because they are usually wrong // on the remote machine (SPARK-12345) (SPARK-25934) private val BLACKLISTED_SPARK_ENV_VARS = Set("SPARK_ENV_LOADED", "SPARK_HOME", "SPARK_CONF_DIR") @@ -424,6 +424,10 @@ private[spark] object RestSubmissionClient { (k.startsWith("SPARK_") && !BLACKLISTED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") } } + + private[spark] def supportsRestClient(master: String): Boolean = { + supportedMasterPrefixes.exists(master.startsWith) + } } private[spark] class RestSubmissionClientApp extends SparkApplication { @@ -456,5 +460,4 @@ private[spark] class RestSubmissionClientApp extends SparkApplication { val env = RestSubmissionClient.filterSystemEnvironment(sys.env) run(appResource, mainClass, appArgs, conf, env) } - } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 22b65abce611a..9c44c995ddfdf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletResponse import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.internal.config import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -132,12 +133,22 @@ private[rest] class StandaloneSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) + // The semantics of "spark.master" and the masterUrl are different. While the + // property "spark.master" could contain all registered masters, masterUrl + // contains only the active master. To make sure a Spark driver can recover + // in a multi-master setup, we use the "spark.master" property while submitting + // the driver. + val masters = sparkProperties.get("spark.master") + val (_, masterPort) = Utils.extractHostPortFromSparkUrl(masterUrl) + val masterRestPort = this.conf.get(config.MASTER_REST_SERVER_PORT) + val updatedMasters = masters.map( + _.replace(s":$masterRestPort", s":$masterPort")).getOrElse(masterUrl) val appArgs = request.appArgs // Filter SPARK_LOCAL_(IP|HOSTNAME) environment variables from being set on the remote system. val environmentVariables = diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 86ddf954ca128..7f462148c71a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.rest import scala.util.Try +import org.apache.spark.internal.config import org.apache.spark.util.Utils /** @@ -49,11 +50,11 @@ private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { assertFieldIsSet(appArgs, "appArgs") assertFieldIsSet(environmentVariables, "environmentVariables") assertPropertyIsSet("spark.app.name") - assertPropertyIsBoolean("spark.driver.supervise") - assertPropertyIsNumeric("spark.driver.cores") - assertPropertyIsNumeric("spark.cores.max") - assertPropertyIsMemory("spark.driver.memory") - assertPropertyIsMemory("spark.executor.memory") + assertPropertyIsBoolean(config.DRIVER_SUPERVISE.key) + assertPropertyIsNumeric(config.DRIVER_CORES.key) + assertPropertyIsNumeric(config.CORES_MAX.key) + assertPropertyIsMemory(config.DRIVER_MEMORY.key) + assertPropertyIsMemory(config.EXECUTOR_MEMORY.key) } private def assertPropertyIsSet(key: String): Unit = diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index ab8d8d96a9b08..10cd8742f2b49 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -17,76 +17,158 @@ package org.apache.spark.deploy.security +import java.io.File +import java.security.PrivilegedExceptionAction +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} +import java.util.concurrent.atomic.AtomicReference + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.ThreadUtils /** - * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to - * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], - * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not - * explicitly disabled. + * Manager for delegation tokens in a Spark application. + * + * This manager has two modes of operation: + * + * 1. When configured with a principal and a keytab, it will make sure long-running apps can run + * without interruption while accessing secured services. It periodically logs in to the KDC with + * user-provided credentials, and contacts all the configured secure services to obtain delegation + * tokens to be distributed to the rest of the application. + * + * Because the Hadoop UGI API does not expose the TTL of the TGT, a configuration controls how often + * to check that a relogin is necessary. This is done reasonably often since the check is a no-op + * when the relogin is not yet needed. The check period can be overridden in the configuration. * - * Also, each HadoopDelegationTokenProvider is controlled by - * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to - * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be - * enabled/disabled by the configuration spark.security.credentials.hive.enabled. + * New delegation tokens are created once 75% of the renewal interval of the original tokens has + * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. + * The driver is tasked with distributing the tokens to other processes that might need them. * - * @param sparkConf Spark configuration - * @param hadoopConf Hadoop configuration - * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. + * 2. When operating without an explicit principal and keytab, token renewal will not be available. + * Starting the manager will distribute an initial set of delegation tokens to the provided Spark + * driver, but the app will not get new tokens when those expire. + * + * It can also be used just to create delegation tokens, by calling the `obtainDelegationTokens` + * method. This option does not require calling the `start` method, but leaves it up to the + * caller to distribute the tokens that were generated. */ private[spark] class HadoopDelegationTokenManager( - sparkConf: SparkConf, - hadoopConf: Configuration, - fileSystems: Configuration => Set[FileSystem]) - extends Logging { + protected val sparkConf: SparkConf, + protected val hadoopConf: Configuration) extends Logging { private val deprecatedProviderEnabledConfigs = List( "spark.yarn.security.tokens.%s.enabled", "spark.yarn.security.credentials.%s.enabled") private val providerEnabledConfig = "spark.security.credentials.%s.enabled" - // Maintain all the registered delegation token providers - private val delegationTokenProviders = getDelegationTokenProviders + private val principal = sparkConf.get(PRINCIPAL).orNull + private val keytab = sparkConf.get(KEYTAB).orNull + + require((principal == null) == (keytab == null), + "Both principal and keytab must be defined, or neither.") + require(keytab == null || new File(keytab).isFile(), s"Cannot find keytab at $keytab.") + + private val delegationTokenProviders = loadProviders() logDebug("Using the following builtin delegation token providers: " + s"${delegationTokenProviders.keys.mkString(", ")}.") - /** Construct a [[HadoopDelegationTokenManager]] for the default Hadoop filesystem */ - def this(sparkConf: SparkConf, hadoopConf: Configuration) = { - this( - sparkConf, - hadoopConf, - hadoopConf => Set(FileSystem.get(hadoopConf).getHomeDirectory.getFileSystem(hadoopConf))) + private var renewalExecutor: ScheduledExecutorService = _ + private val driverRef = new AtomicReference[RpcEndpointRef]() + + /** Set the endpoint used to send tokens to the driver. */ + def setDriverRef(ref: RpcEndpointRef): Unit = { + driverRef.set(ref) } - private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { - val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++ - safeCreateProvider(new HiveDelegationTokenProvider) ++ - safeCreateProvider(new HBaseDelegationTokenProvider) + /** @return Whether delegation token renewal is enabled. */ + def renewalEnabled: Boolean = principal != null - // Filter out providers for which spark.security.credentials.{service}.enabled is false. - providers - .filter { p => isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap + /** + * Start the token renewer. Requires a principal and keytab. Upon start, the renewer will: + * + * - log in the configured principal, and set up a task to keep that user's ticket renewed + * - obtain delegation tokens from all available providers + * - send the tokens to the driver, if it's already registered + * - schedule a periodic task to update the tokens when needed. + * + * @return The newly logged in user. + */ + def start(): UserGroupInformation = { + require(renewalEnabled, "Token renewal must be enabled to start the renewer.") + renewalExecutor = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") + + val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() + val ugi = doLogin() + + val tgtRenewalTask = new Runnable() { + override def run(): Unit = { + ugi.checkTGTAndReloginFromKeytab() + } + } + val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) + renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, + TimeUnit.SECONDS) + + val creds = obtainTokensAndScheduleRenewal(ugi) + ugi.addCredentials(creds) + + val driver = driverRef.get() + if (driver != null) { + val tokens = SparkHadoopUtil.get.serialize(creds) + driver.send(UpdateDelegationTokens(tokens)) + } + + // Transfer the original user's tokens to the new user, since it may contain needed tokens + // (such as those user to connect to YARN). Explicitly avoid overwriting tokens that already + // exist in the current user's credentials, since those were freshly obtained above + // (see SPARK-23361). + val existing = ugi.getCredentials() + existing.mergeAll(originalCreds) + ugi.addCredentials(existing) + ugi } - private def safeCreateProvider( - createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { - try { - Some(createFn) - } catch { - case t: Throwable => - logDebug(s"Failed to load built in provider.", t) - None + def stop(): Unit = { + if (renewalExecutor != null) { + renewalExecutor.shutdown() } } - def isServiceEnabled(serviceName: String): Boolean = { + /** + * Fetch new delegation tokens for configured services, storing them in the given credentials. + * Tokens are fetched for the current logged in user. + * + * @param creds Credentials object where to store the delegation tokens. + * @return The time by which the tokens must be renewed. + */ + def obtainDelegationTokens(creds: Credentials): Long = { + delegationTokenProviders.values.flatMap { provider => + if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { + provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") + None + } + }.foldLeft(Long.MaxValue)(math.min) + } + + // Visible for testing. + def isProviderLoaded(serviceName: String): Boolean = { + delegationTokenProviders.contains(serviceName) + } + + protected def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) deprecatedProviderEnabledConfigs.foreach { pattern => @@ -110,32 +192,104 @@ private[spark] class HadoopDelegationTokenManager( } /** - * Get delegation token provider for the specified service. + * List of file systems for which to obtain delegation tokens. The base implementation + * returns just the default file system in the given Hadoop configuration. */ - def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { - delegationTokenProviders.get(service) + protected def fileSystemsToAccess(): Set[FileSystem] = { + Set(FileSystem.get(hadoopConf)) + } + + private def scheduleRenewal(delay: Long): Unit = { + val _delay = math.max(0, delay) + logInfo(s"Scheduling login from keytab in ${UIUtils.formatDuration(delay)}.") + + val renewalTask = new Runnable() { + override def run(): Unit = { + updateTokensTask() + } + } + renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) } /** - * Writes delegation tokens to creds. Delegation tokens are fetched from all registered - * providers. - * - * @param hadoopConf hadoop Configuration - * @param creds Credentials that will be updated in place (overwritten) - * @return Time after which the fetched delegation tokens should be renewed. + * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself + * to fetch the next set of tokens when needed. */ - def obtainDelegationTokens( - hadoopConf: Configuration, - creds: Credentials): Long = { - delegationTokenProviders.values.flatMap { provider => - if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { - provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + private def updateTokensTask(): Unit = { + try { + val freshUGI = doLogin() + val creds = obtainTokensAndScheduleRenewal(freshUGI) + val tokens = SparkHadoopUtil.get.serialize(creds) + + val driver = driverRef.get() + if (driver != null) { + logInfo("Updating delegation tokens.") + driver.send(UpdateDelegationTokens(tokens)) } else { - logDebug(s"Service ${provider.serviceName} does not require a token." + - s" Check your configuration to see if security is disabled or not.") - None + // This shouldn't really happen, since the driver should register way before tokens expire. + logWarning("Delegation tokens close to expiration but no driver has registered yet.") + SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) } - }.foldLeft(Long.MaxValue)(math.min) + } catch { + case e: Exception => + val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) + logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + + " If this happens too often tasks will fail.", e) + scheduleRenewal(delay) + } + } + + /** + * Obtain new delegation tokens from the available providers. Schedules a new task to fetch + * new tokens before the new set expires. + * + * @return Credentials containing the new tokens. + */ + private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { + ugi.doAs(new PrivilegedExceptionAction[Credentials]() { + override def run(): Credentials = { + val creds = new Credentials() + val nextRenewal = obtainDelegationTokens(creds) + + // Calculate the time when new credentials should be created, based on the configured + // ratio. + val now = System.currentTimeMillis + val ratio = sparkConf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) + val delay = (ratio * (nextRenewal - now)).toLong + scheduleRenewal(delay) + creds + } + }) + } + + private def doLogin(): UserGroupInformation = { + logInfo(s"Attempting to login to KDC using principal: $principal") + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + logInfo("Successfully logged into KDC.") + ugi + } + + private def loadProviders(): Map[String, HadoopDelegationTokenProvider] = { + val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystemsToAccess)) ++ + safeCreateProvider(new HiveDelegationTokenProvider) ++ + safeCreateProvider(new HBaseDelegationTokenProvider) + + // Filter out providers for which spark.security.credentials.{service}.enabled is false. + providers + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap } -} + private def safeCreateProvider( + createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { + try { + Some(createFn) + } catch { + case t: Throwable => + logDebug(s"Failed to load built in provider.", t) + None + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 21ca669ea98f0..767b5521e8d7b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration => Set[FileSystem]) +private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => Set[FileSystem]) extends HadoopDelegationTokenProvider with Logging { // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. @@ -44,8 +44,7 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] = { - - val fsToGetTokens = fileSystems(hadoopConf) + val fsToGetTokens = fileSystems() val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) // Get the token renewal interval if it is not set. It will only be called once. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index a6d13d12fc28d..8c2a907b86890 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Worker.WORKER_DRIVER_TERMINATE_TIMEOUT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, Utils} @@ -57,8 +58,7 @@ private[deploy] class DriverRunner( @volatile private[worker] var finalException: Option[Exception] = None // Timeout to wait for when trying to terminate a driver. - private val DRIVER_TERMINATE_TIMEOUT_MS = - conf.getTimeAsMs("spark.worker.driverTerminateTimeout", "10s") + private val driverTerminateTimeoutMs = conf.get(WORKER_DRIVER_TERMINATE_TIMEOUT) // Decoupled for testing def setClock(_clock: Clock): Unit = { @@ -122,7 +122,7 @@ private[deploy] class DriverRunner( killed = true synchronized { process.foreach { p => - val exitCode = Utils.terminateProcess(p, DRIVER_TERMINATE_TIMEOUT_MS) + val exitCode = Utils.terminateProcess(p, driverTerminateTimeoutMs) if (exitCode.isEmpty) { logWarning("Failed to terminate driver process: " + p + ". This process will likely be orphaned.") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 8d6a2b80ef5f2..1e8ad0b6af6a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -23,7 +23,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil, SparkSubmit} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util._ @@ -43,7 +43,7 @@ object DriverWrapper extends Logging { case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() val host: String = Utils.localHostName() - val port: Int = sys.props.getOrElse("spark.driver.port", "0").toInt + val port: Int = sys.props.getOrElse(config.DRIVER_PORT.key, "0").toInt val rpcEnv = RpcEnv.create("Driver", host, port, conf, new SecurityManager(conf)) logInfo(s"Driver address: ${rpcEnv.address}") rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) @@ -51,7 +51,7 @@ object DriverWrapper extends Logging { val currentLoader = Thread.currentThread.getContextClassLoader val userJarUrl = new File(userJar).toURI().toURL() val loader = - if (sys.props.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { + if (sys.props.getOrElse(config.DRIVER_USER_CLASS_PATH_FIRST.key, "false").toBoolean) { new ChildFirstURLClassLoader(Array(userJarUrl), currentLoader) } else { new MutableURLClassLoader(Array(userJarUrl), currentLoader) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index dc6a3076a5113..c74a95718d82e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -28,6 +28,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender @@ -160,7 +161,7 @@ private[deploy] class ExecutorRunner( // Add webUI log urls val baseUrl = - if (conf.getBoolean("spark.ui.reverseProxy", false)) { + if (conf.get(UI_REVERSE_PROXY)) { s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType=" } else { s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index d5ea2523c628b..4b9dc03676473 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -37,6 +37,8 @@ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -72,7 +74,7 @@ private[deploy] class Worker( // For worker and executor IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.get(WORKER_TIMEOUT) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) @@ -91,13 +93,11 @@ private[deploy] class Worker( private val PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)) - private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.get(WORKER_CLEANUP_ENABLED) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = - conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = conf.get(WORKER_CLEANUP_INTERVAL) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECONDS = - conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + private val APP_DATA_RETENTION_SECONDS = conf.get(APP_DATA_RETENTION) // Whether or not cleanup the non-shuffle files on executor exits. private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = @@ -110,8 +110,7 @@ private[deploy] class Worker( * Whether to use the master address in `masterRpcAddresses` if possible. If it's disabled, Worker * will just use the address received from Master. */ - private val preferConfiguredMasterAddress = - conf.getBoolean("spark.worker.preferConfiguredMasterAddress", false) + private val preferConfiguredMasterAddress = conf.get(PREFER_CONFIGURED_MASTER_ADDRESS) /** * The master address to connect in case of failure. When the connection is broken, worker will * use this address to connect. This is usually just one of `masterRpcAddresses`. However, when @@ -142,10 +141,8 @@ private[deploy] class Worker( val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] - val retainedExecutors = conf.getInt("spark.worker.ui.retainedExecutors", - WorkerWebUI.DEFAULT_RETAINED_EXECUTORS) - val retainedDrivers = conf.getInt("spark.worker.ui.retainedDrivers", - WorkerWebUI.DEFAULT_RETAINED_DRIVERS) + val retainedExecutors = conf.get(WORKER_UI_RETAINED_EXECUTORS) + val retainedDrivers = conf.get(WORKER_UI_RETAINED_DRIVERS) // The shuffle service is not actually started unless configured. private val shuffleService = if (externalShuffleServiceSupplier != null) { @@ -165,7 +162,7 @@ private[deploy] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + val reverseProxy = conf.get(UI_REVERSE_PROXY) private var registerMasterFutures: Array[JFuture[_]] = null private var registrationRetryTimer: Option[JScheduledFuture[_]] = None diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 580281288b060..8c87708e960e6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory import scala.annotation.tailrec import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Worker._ import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** @@ -59,9 +60,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.worker.ui.port")) { - webUiPort = conf.get("spark.worker.ui.port").toInt - } + conf.get(WORKER_UI_PORT).foreach { webUiPort = _ } checkWorkerMemory() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index ea67b7434a769..f354f18f15623 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,6 +57,4 @@ class WorkerWebUI( private[worker] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_RETAINED_DRIVERS = 1000 - val DEFAULT_RETAINED_EXECUTORS = 1000 } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala new file mode 100644 index 0000000000000..ceab957b36634 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +private[spark] object Deploy { + val RECOVERY_MODE = ConfigBuilder("spark.deploy.recoveryMode") + .stringConf + .createWithDefault("NONE") + + val RECOVERY_MODE_FACTORY = ConfigBuilder("spark.deploy.recoveryMode.factory") + .stringConf + .createWithDefault("") + + val RECOVERY_DIRECTORY = ConfigBuilder("spark.deploy.recoveryDirectory") + .stringConf + .createWithDefault("") + + val ZOOKEEPER_URL = ConfigBuilder("spark.deploy.zookeeper.url") + .doc(s"When `${RECOVERY_MODE.key}` is set to ZOOKEEPER, this " + + "configuration is used to set the zookeeper URL to connect to.") + .stringConf + .createOptional + + val ZOOKEEPER_DIRECTORY = ConfigBuilder("spark.deploy.zookeeper.dir") + .stringConf + .createOptional + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.deploy.retainedApplications") + .intConf + .createWithDefault(200) + + val RETAINED_DRIVERS = ConfigBuilder("spark.deploy.retainedDrivers") + .intConf + .createWithDefault(200) + + val REAPER_ITERATIONS = ConfigBuilder("spark.dead.worker.persistence") + .intConf + .createWithDefault(15) + + val MAX_EXECUTOR_RETRIES = ConfigBuilder("spark.deploy.maxExecutorRetries") + .intConf + .createWithDefault(10) + + val SPREAD_OUT_APPS = ConfigBuilder("spark.deploy.spreadOut") + .booleanConf + .createWithDefault(true) + + val DEFAULT_CORES = ConfigBuilder("spark.deploy.defaultCores") + .intConf + .createWithDefault(Int.MaxValue) + + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/config.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/deploy/history/config.scala rename to core/src/main/scala/org/apache/spark/internal/config/History.scala index 25ba9edb9e014..571eac6ae08c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/config.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.deploy.history +package org.apache.spark.internal.config import java.util.concurrent.TimeUnit -import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -private[spark] object config { +private[spark] object History { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" @@ -64,4 +63,44 @@ private[spark] object config { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("1m") + val MAX_DRIVER_LOG_AGE_S = ConfigBuilder("spark.history.fs.driverlog.cleaner.maxAge") + .fallbackConf(MAX_LOG_AGE_S) + + val HISTORY_SERVER_UI_ACLS_ENABLE = ConfigBuilder("spark.history.ui.acls.enable") + .booleanConf + .createWithDefault(false) + + val HISTORY_SERVER_UI_ADMIN_ACLS = ConfigBuilder("spark.history.ui.admin.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS = ConfigBuilder("spark.history.ui.admin.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val NUM_REPLAY_THREADS = ConfigBuilder("spark.history.fs.numReplayThreads") + .intConf + .createWithDefaultFunction(() => Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.history.retainedApplications") + .intConf + .createWithDefault(50) + + val PROVIDER = ConfigBuilder("spark.history.provider") + .stringConf + .createOptional + + val KERBEROS_ENABLED = ConfigBuilder("spark.history.kerberos.enabled") + .booleanConf + .createWithDefault(false) + + val KERBEROS_PRINCIPAL = ConfigBuilder("spark.history.kerberos.principal") + .stringConf + .createOptional + + val KERBEROS_KEYTAB = ConfigBuilder("spark.history.kerberos.keytab") + .stringConf + .createOptional } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala new file mode 100644 index 0000000000000..7873141440dec --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Kryo { + + val KRYO_REGISTRATION_REQUIRED = ConfigBuilder("spark.kryo.registrationRequired") + .booleanConf + .createWithDefault(false) + + val KRYO_USER_REGISTRATORS = ConfigBuilder("spark.kryo.registrator") + .stringConf + .createOptional + + val KRYO_CLASSES_TO_REGISTER = ConfigBuilder("spark.kryo.classesToRegister") + .stringConf + .toSequence + .createWithDefault(Nil) + + val KRYO_USE_UNSAFE = ConfigBuilder("spark.kyro.unsafe") + .booleanConf + .createWithDefault(false) + + val KRYO_USE_POOL = ConfigBuilder("spark.kyro.pool") + .booleanConf + .createWithDefault(true) + + val KRYO_REFERENCE_TRACKING = ConfigBuilder("spark.kryo.referenceTracking") + .booleanConf + .createWithDefault(true) + + val KRYO_SERIALIZER_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("64k") + + val KRYO_SERIALIZER_MAX_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer.max") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("64m") + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Python.scala b/core/src/main/scala/org/apache/spark/internal/config/Python.scala new file mode 100644 index 0000000000000..26a0598f49411 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Python.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Python { + val PYTHON_WORKER_REUSE = ConfigBuilder("spark.python.worker.reuse") + .booleanConf + .createWithDefault(true) + + val PYTHON_TASK_KILL_TIMEOUT = ConfigBuilder("spark.python.task.killTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("2s") + + val PYTHON_USE_DAEMON = ConfigBuilder("spark.python.use.daemon") + .booleanConf + .createWithDefault(true) + + val PYTHON_DAEMON_MODULE = ConfigBuilder("spark.python.daemon.module") + .stringConf + .createOptional + + val PYTHON_WORKER_MODULE = ConfigBuilder("spark.python.worker.module") + .stringConf + .createOptional + + val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") + .bytesConf(ByteUnit.MiB) + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/R.scala b/core/src/main/scala/org/apache/spark/internal/config/R.scala new file mode 100644 index 0000000000000..26e06a5231c42 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/R.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.internal.config + +private[spark] object R { + + val R_BACKEND_CONNECTION_TIMEOUT = ConfigBuilder("spark.r.backendConnectionTimeout") + .intConf + .createWithDefault(6000) + + val R_NUM_BACKEND_THREADS = ConfigBuilder("spark.r.numRBackendThreads") + .intConf + .createWithDefault(2) + + val R_HEARTBEAT_INTERVAL = ConfigBuilder("spark.r.heartBeatInterval") + .intConf + .createWithDefault(100) + + val SPARKR_COMMAND = ConfigBuilder("spark.sparkr.r.command") + .stringConf + .createWithDefault("Rscript") + + val R_COMMAND = ConfigBuilder("spark.r.command") + .stringConf + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/status/config.scala b/core/src/main/scala/org/apache/spark/internal/config/Status.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/status/config.scala rename to core/src/main/scala/org/apache/spark/internal/config/Status.scala index 67801b8f046f4..c56157227f8fc 100644 --- a/core/src/main/scala/org/apache/spark/status/config.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Status.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.status +package org.apache.spark.internal.config import java.util.concurrent.TimeUnit -import org.apache.spark.internal.config._ - -private[spark] object config { +private[spark] object Status { val ASYNC_TRACKING_ENABLED = ConfigBuilder("spark.appStateStore.asyncTracking.enable") .booleanConf @@ -51,4 +49,10 @@ private[spark] object config { .intConf .createWithDefault(Int.MaxValue) + val APP_STATUS_METRICS_ENABLED = + ConfigBuilder("spark.app.status.metrics.enabled") + .doc("Whether Dropwizard/Codahale metrics " + + "will be reported for the status of the running spark app.") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala new file mode 100644 index 0000000000000..21660ab3a9512 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +private[spark] object Tests { + + val TEST_USE_COMPRESSED_OOPS_KEY = "spark.test.useCompressedOops" + + val TEST_MEMORY = ConfigBuilder("spark.testing.memory") + .longConf + .createWithDefault(Runtime.getRuntime.maxMemory) + + val TEST_SCHEDULE_INTERVAL = + ConfigBuilder("spark.testing.dynamicAllocation.scheduleInterval") + .longConf + .createWithDefault(100) + + val IS_TESTING = ConfigBuilder("spark.testing") + .booleanConf + .createOptional + + val TEST_NO_STAGE_RETRY = ConfigBuilder("spark.test.noStageRetry") + .booleanConf + .createWithDefault(false) + + val TEST_RESERVED_MEMORY = ConfigBuilder("spark.testing.reservedMemory") + .longConf + .createOptional + + val TEST_N_HOSTS = ConfigBuilder("spark.testing.nHosts") + .intConf + .createWithDefault(5) + + val TEST_N_EXECUTORS_HOST = ConfigBuilder("spark.testing.nExecutorsPerHost") + .intConf + .createWithDefault(4) + + val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") + .intConf + .createWithDefault(2) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala new file mode 100644 index 0000000000000..6c04f0dd2bbb3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object UI { + + val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") + .doc("When true, show the progress bar in the console.") + .booleanConf + .createWithDefault(false) + + val UI_CONSOLE_PROGRESS_UPDATE_INTERVAL = + ConfigBuilder("spark.ui.consoleProgress.update.interval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(200) + + val UI_ENABLED = ConfigBuilder("spark.ui.enabled") + .doc("Whether to run the web UI for the Spark application.") + .booleanConf + .createWithDefault(true) + + val UI_PORT = ConfigBuilder("spark.ui.port") + .doc("Port for your application's dashboard, which shows memory and workload data.") + .intConf + .createWithDefault(4040) + + val UI_FILTERS = ConfigBuilder("spark.ui.filters") + .doc("Comma separated list of filter class names to apply to the Spark Web UI.") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_ALLOW_FRAMING_FROM = ConfigBuilder("spark.ui.allowFramingFrom") + .stringConf + .createOptional + + val UI_REVERSE_PROXY = ConfigBuilder("spark.ui.reverseProxy") + .doc("Enable running Spark Master as reverse proxy for worker and application UIs. " + + "In this mode, Spark master will reverse proxy the worker and application UIs to enable " + + "access without requiring direct access to their hosts. Use it with caution, as worker " + + "and application UI will not be accessible directly, you will only be able to access them" + + "through spark master/proxy public URL. This setting affects all the workers and " + + "application UIs running in the cluster and must be set on all the workers, drivers " + + " and masters.") + .booleanConf + .createWithDefault(false) + + val UI_REVERSE_PROXY_URL = ConfigBuilder("spark.ui.reverseProxyUrl") + .doc("This is the URL where your proxy is running. This URL is for proxy which is running " + + "in front of Spark Master. This is useful when running proxy for authentication e.g. " + + "OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to " + + "reach your proxy.") + .stringConf + .createOptional + + val UI_KILL_ENABLED = ConfigBuilder("spark.ui.killEnabled") + .doc("Allows jobs and stages to be killed from the web UI.") + .booleanConf + .createWithDefault(true) + + val UI_THREAD_DUMPS_ENABLED = ConfigBuilder("spark.ui.threadDumpsEnabled") + .booleanConf + .createWithDefault(true) + + val UI_X_XSS_PROTECTION = ConfigBuilder("spark.ui.xXssProtection") + .doc("Value for HTTP X-XSS-Protection response header") + .stringConf + .createWithDefaultString("1; mode=block") + + val UI_X_CONTENT_TYPE_OPTIONS = ConfigBuilder("spark.ui.xContentTypeOptions.enabled") + .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") + .booleanConf + .createWithDefault(true) + + val UI_STRICT_TRANSPORT_SECURITY = ConfigBuilder("spark.ui.strictTransportSecurity") + .doc("Value for HTTP Strict Transport Security Response Header") + .stringConf + .createOptional + + val UI_REQUEST_HEADER_SIZE = ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + + val UI_TIMELINE_TASKS_MAXIMUM = ConfigBuilder("spark.ui.timeline.tasks.maximum") + .intConf + .createWithDefault(1000) + + val ACLS_ENABLE = ConfigBuilder("spark.acls.enable") + .booleanConf + .createWithDefault(false) + + val UI_VIEW_ACLS = ConfigBuilder("spark.ui.view.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_VIEW_ACLS_GROUPS = ConfigBuilder("spark.ui.view.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS = ConfigBuilder("spark.admin.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS_GROUPS = ConfigBuilder("spark.admin.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS = ConfigBuilder("spark.modify.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS_GROUPS = ConfigBuilder("spark.modify.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val USER_GROUPS_MAPPING = ConfigBuilder("spark.user.groups.mapping") + .stringConf + .createWithDefault("org.apache.spark.security.ShellBasedGroupsMappingProvider") +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala new file mode 100644 index 0000000000000..47f7167d2c9cb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +private[spark] object Worker { + val WORKER_TIMEOUT = ConfigBuilder("spark.worker.timeout") + .longConf + .createWithDefault(60) + + val WORKER_DRIVER_TERMINATE_TIMEOUT = ConfigBuilder("spark.worker.driverTerminateTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10s") + + val WORKER_CLEANUP_ENABLED = ConfigBuilder("spark.worker.cleanup.enabled") + .booleanConf + .createWithDefault(false) + + val WORKER_CLEANUP_INTERVAL = ConfigBuilder("spark.worker.cleanup.interval") + .longConf + .createWithDefault(60 * 30) + + val APP_DATA_RETENTION = ConfigBuilder("spark.worker.cleanup.appDataTtl") + .longConf + .createWithDefault(7 * 24 * 3600) + + val PREFER_CONFIGURED_MASTER_ADDRESS = ConfigBuilder("spark.worker.preferConfiguredMasterAddress") + .booleanConf + .createWithDefault(false) + + val WORKER_UI_PORT = ConfigBuilder("spark.worker.ui.port") + .intConf + .createOptional + + val WORKER_UI_RETAINED_EXECUTORS = ConfigBuilder("spark.worker.ui.retainedExecutors") + .intConf + .createWithDefault(1000) + + val WORKER_UI_RETAINED_DRIVERS = ConfigBuilder("spark.worker.ui.retainedDrivers") + .intConf + .createWithDefault(1000) + + val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + ConfigBuilder("spark.worker.ui.compressedLogFileLengthCacheSize") + .intConf + .createWithDefault(100) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5c17b9b3a3207..67e4449d1690c 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -38,7 +38,12 @@ package object config { private[spark] val DRIVER_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory") + private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") + .doc("Number of cores to use for the driver process, only in cluster mode.") + .intConf + .createWithDefault(1) + + private[spark] val DRIVER_MEMORY = ConfigBuilder(SparkLauncher.DRIVER_MEMORY) .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") @@ -76,6 +81,9 @@ package object config { private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EXECUTOR_ID = + ConfigBuilder("spark.executor.id").stringConf.createOptional + private[spark] val EXECUTOR_CLASS_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional @@ -88,7 +96,11 @@ package object config { private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory") + private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) + .intConf + .createWithDefault(1) + + private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") @@ -99,6 +111,15 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val CORES_MAX = ConfigBuilder("spark.cores.max") + .doc("When running on a standalone deploy cluster or a Mesos cluster in coarse-grained " + + "sharing mode, the maximum amount of CPU cores to request for the application from across " + + "the cluster (not from each machine). If not set, the default will be " + + "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + + "(all available cores) on Mesos.") + .intConf + .createOptional + private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") @@ -115,10 +136,6 @@ package object config { .checkValue(_ >= 0, "The off-heap memory size must not be negative") .createWithDefault(0) - private[spark] val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") - .bytesConf(ByteUnit.MiB) - .createOptional - private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) @@ -156,6 +173,10 @@ package object config { .doc("Name of the Kerberos principal.") .stringConf.createOptional + private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.kerberos.relogin.period") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("1m") + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") .intConf .createOptional @@ -254,6 +275,10 @@ package object config { .stringConf .createOptional + private[spark] val METRICS_CONF = ConfigBuilder("spark.metrics.conf") + .stringConf + .createOptional + private[spark] val PYSPARK_DRIVER_PYTHON = ConfigBuilder("spark.pyspark.driver.python") .stringConf .createOptional @@ -266,11 +291,6 @@ package object config { private[spark] val HISTORY_UI_MAX_APPS = ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) - private[spark] val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") - .doc("When true, show the progress bar in the console.") - .booleanConf - .createWithDefault(false) - private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled") .booleanConf .createWithDefault(false) @@ -296,6 +316,17 @@ package object config { .stringConf .createWithDefault(Utils.localCanonicalHostName()) + private[spark] val DRIVER_PORT = ConfigBuilder("spark.driver.port") + .doc("Port of driver endpoints.") + .intConf + .createWithDefault(0) + + private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise") + .doc("If true, restarts the driver automatically if it fails with a non-zero exit status. " + + "Only has effect in Spark standalone mode or Mesos cluster deploy mode.") + .booleanConf + .createWithDefault(false) + private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") .doc("Address where to bind network listen sockets on the driver.") .fallbackConf(DRIVER_HOST_ADDRESS) @@ -363,6 +394,11 @@ package object config { .regexConf .createOptional + private[spark] val AUTH_SECRET = + ConfigBuilder("spark.authenticate.secret") + .stringConf + .createOptional + private[spark] val AUTH_SECRET_BIT_LENGTH = ConfigBuilder("spark.authenticate.secretBitLength") .intConf @@ -378,6 +414,37 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val AUTH_SECRET_FILE = + ConfigBuilder("spark.authenticate.secret.file") + .doc("Path to a file that contains the authentication secret to use. The secret key is " + + "loaded from this path on both the driver and the executors if overrides are not set for " + + "either entity (see below). File-based secret keys are only allowed when using " + + "Kubernetes.") + .stringConf + .createOptional + + private[spark] val AUTH_SECRET_FILE_DRIVER = + ConfigBuilder("spark.authenticate.secret.driver.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "driver. In Kubernetes client mode it is often useful to set a different secret " + + "path for the driver vs. the executors, since the driver may not be running in " + + "a pod unlike the executors. If this is set, an accompanying secret file must " + + "be specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) + + private[spark] val AUTH_SECRET_FILE_EXECUTOR = + ConfigBuilder("spark.authenticate.secret.executor.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "executors only. In Kubernetes client mode it is often useful to set a different " + + "secret path for the driver vs. the executors, since the driver may not be running " + + "in a pod unlike the executors. If this is set, an accompanying secret file must be " + + "specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) + private[spark] val NETWORK_ENCRYPTION_ENABLED = ConfigBuilder("spark.network.crypto.enabled") .booleanConf @@ -510,30 +577,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val UI_X_XSS_PROTECTION = - ConfigBuilder("spark.ui.xXssProtection") - .doc("Value for HTTP X-XSS-Protection response header") - .stringConf - .createWithDefaultString("1; mode=block") - - private[spark] val UI_X_CONTENT_TYPE_OPTIONS = - ConfigBuilder("spark.ui.xContentTypeOptions.enabled") - .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") - .booleanConf - .createWithDefault(true) - - private[spark] val UI_STRICT_TRANSPORT_SECURITY = - ConfigBuilder("spark.ui.strictTransportSecurity") - .doc("Value for HTTP Strict Transport Security Response Header") - .stringConf - .createOptional - - private[spark] val UI_REQUEST_HEADER_SIZE = - ConfigBuilder("spark.ui.requestHeaderSize") - .doc("Value for HTTP request header size in bytes.") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("8k") - private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf @@ -646,4 +689,178 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val EXECUTOR_LOGS_ROLLING_STRATEGY = + ConfigBuilder("spark.executor.logs.rolling.strategy").stringConf.createWithDefault("") + + private[spark] val EXECUTOR_LOGS_ROLLING_TIME_INTERVAL = + ConfigBuilder("spark.executor.logs.rolling.time.interval").stringConf.createWithDefault("daily") + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_SIZE = + ConfigBuilder("spark.executor.logs.rolling.maxSize") + .stringConf + .createWithDefault((1024 * 1024).toString) + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES = + ConfigBuilder("spark.executor.logs.rolling.maxRetainedFiles").intConf.createWithDefault(-1) + + private[spark] val EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION = + ConfigBuilder("spark.executor.logs.rolling.enableCompression") + .booleanConf + .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_ENABLED = ConfigBuilder("spark.master.rest.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_PORT = ConfigBuilder("spark.master.rest.port") + .intConf + .createWithDefault(6066) + + private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") + .intConf + .createWithDefault(8080) + + private[spark] val IO_COMPRESSION_SNAPPY_BLOCKSIZE = + ConfigBuilder("spark.io.compression.snappy.blockSize") + .doc("Block size in bytes used in Snappy compression, in the case when " + + "Snappy compression codec is used. Lowering this block size " + + "will also lower shuffle memory usage when Snappy is used") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_LZ4_BLOCKSIZE = + ConfigBuilder("spark.io.compression.lz4.blockSize") + .doc("Block size in bytes used in LZ4 compression, in the case when LZ4 compression" + + "codec is used. Lowering this block size will also lower shuffle memory " + + "usage when LZ4 is used.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_CODEC = + ConfigBuilder("spark.io.compression.codec") + .doc("The codec used to compress internal data such as RDD partitions, event log, " + + "broadcast variables and shuffle outputs. By default, Spark provides four codecs: " + + "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + + "the codec") + .stringConf + .createWithDefaultString("lz4") + + private[spark] val IO_COMPRESSION_ZSTD_BUFFERSIZE = + ConfigBuilder("spark.io.compression.zstd.bufferSize") + .doc("Buffer size in bytes used in Zstd compression, in the case when Zstd " + + "compression codec is used. Lowering this size will lower the shuffle " + + "memory usage when Zstd is used, but it might increase the compression " + + "cost because of excessive JNI call overhead") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_ZSTD_LEVEL = + ConfigBuilder("spark.io.compression.zstd.level") + .doc("Compression level for Zstd compression codec. Increasing the compression " + + "level will result in better compression at the expense of more CPU and memory") + .intConf + .createWithDefault(1) + + private[spark] val BUFFER_SIZE = + ConfigBuilder("spark.buffer.size") + .intConf + .createWithDefault(65536) + + private[spark] val LOCALITY_WAIT_PROCESS = ConfigBuilder("spark.locality.wait.process") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_NODE = ConfigBuilder("spark.locality.wait.node") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") + .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + + "in MiB unless otherwise specified. Since each output requires us to create a " + + "buffer to receive it, this represents a fixed memory overhead per reduce task, " + + "so keep it small unless you have a large amount of memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("48m") + + private[spark] val REDUCER_MAX_REQS_IN_FLIGHT = ConfigBuilder("spark.reducer.maxReqsInFlight") + .doc("This configuration limits the number of remote requests to fetch blocks at " + + "any given point. When the number of hosts in the cluster increase, " + + "it might lead to very large number of inbound connections to one or more nodes, " + + "causing the workers to fail under load. By allowing it to limit the number of " + + "fetch requests, this scenario can be mitigated") + .intConf + .createWithDefault(Int.MaxValue) + + private[spark] val BROADCAST_COMPRESS = ConfigBuilder("spark.broadcast.compress") + .doc("Whether to compress broadcast variables before sending them. " + + "Generally a good idea. Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(true) + + private[spark] val BROADCAST_BLOCKSIZE = ConfigBuilder("spark.broadcast.blockSize") + .doc("Size of each piece of a block for TorrentBroadcastFactory, in " + + "KiB unless otherwise specified. Too large a value decreases " + + "parallelism during broadcast (makes it slower); however, " + + "if it is too small, BlockManager might take a performance hit") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("4m") + + private[spark] val BROADCAST_CHECKSUM = ConfigBuilder("spark.broadcast.checksum") + .doc("Whether to enable checksum for broadcast. If enabled, " + + "broadcasts will include a checksum, which can help detect " + + "corrupted blocks, at the cost of computing and sending a little " + + "more data. It's possible to disable it if the network has other " + + "mechanisms to guarantee data won't be corrupted during broadcast") + .booleanConf.createWithDefault(true) + + private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") + .doc("Whether to compress serialized RDD partitions " + + "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + + "or StorageLevel.MEMORY_ONLY in Python). Can save substantial " + + "space at the cost of some extra CPU time. " + + "Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(false) + + private[spark] val RDD_PARALLEL_LISTING_THRESHOLD = + ConfigBuilder("spark.rdd.parallelListingThreshold") + .intConf + .createWithDefault(10) + + private[spark] val RDD_LIMIT_SCALE_UP_FACTOR = + ConfigBuilder("spark.rdd.limit.scaleUpFactor") + .intConf + .createWithDefault(4) + + private[spark] val SERIALIZER = ConfigBuilder("spark.serializer") + .stringConf + .createWithDefault("org.apache.spark.serializer.JavaSerializer") + + private[spark] val SERIALIZER_OBJECT_STREAM_RESET = + ConfigBuilder("spark.serializer.objectStreamReset") + .intConf + .createWithDefault(100) + + private[spark] val SERIALIZER_EXTRA_DEBUG_INFO = ConfigBuilder("spark.serializer.extraDebugInfo") + .booleanConf + .createWithDefault(true) + + private[spark] val JARS = ConfigBuilder("spark.jars") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val FILES = ConfigBuilder("spark.files") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode") + .stringConf + .createWithDefault("client") + + private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles") + .stringConf + .toSequence + .createWithDefault(Nil) } diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index a6f7db0600e60..8286087042741 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.storage.BlockId /** @@ -127,14 +128,14 @@ private[spark] object StaticMemoryManager { if (systemMaxMemory < MIN_MEMORY_BYTES) { throw new IllegalArgumentException(s"System memory $systemMaxMemory must " + s"be at least $MIN_MEMORY_BYTES. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") + s"option or ${config.DRIVER_MEMORY.key} in Spark configuration.") } - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (conf.contains(config.EXECUTOR_MEMORY)) { + val executorMemory = conf.getSizeAsBytes(config.EXECUTOR_MEMORY.key) if (executorMemory < MIN_MEMORY_BYTES) { throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + s"$MIN_MEMORY_BYTES. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") + s"--executor-memory option or ${config.EXECUTOR_MEMORY.key} in Spark configuration.") } } val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 78edd2c4d7faa..9260fd3a6fb34 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.memory import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.storage.BlockId /** @@ -216,15 +217,15 @@ object UnifiedMemoryManager { if (systemMemory < minSystemMemory) { throw new IllegalArgumentException(s"System memory $systemMemory must " + s"be at least $minSystemMemory. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") + s"option or ${config.DRIVER_MEMORY.key} in Spark configuration.") } // SPARK-12759 Check executor memory to fail fast if memory is insufficient - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (conf.contains(config.EXECUTOR_MEMORY)) { + val executorMemory = conf.getSizeAsBytes(config.EXECUTOR_MEMORY.key) if (executorMemory < minSystemMemory) { throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + s"$minSystemMemory. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") + s"--executor-memory option or ${config.EXECUTOR_MEMORY.key} in Spark configuration.") } } val usableMemory = systemMemory - reservedMemory diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index a4056508c181e..b6be8aaefd351 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -26,6 +26,7 @@ import scala.util.matching.Regex import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.util.Utils private[spark] class MetricsConfig(conf: SparkConf) extends Logging { @@ -52,7 +53,7 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { // Add default properties in case there's no properties file setDefaultProperties(properties) - loadPropertiesFromFile(conf.getOption("spark.metrics.conf")) + loadPropertiesFromFile(conf.get(METRICS_CONF)) // Also look for the properties in provided Spark configuration val prefix = "spark.metrics.conf." diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 3457a2632277d..c6c3ac242f4ec 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -128,7 +128,7 @@ private[spark] class MetricsSystem private ( private[spark] def buildRegistryName(source: Source): String = { val metricsNamespace = conf.get(METRICS_NAMESPACE).orElse(conf.getOption("spark.app.id")) - val executorId = conf.getOption("spark.executor.id") + val executorId = conf.get(EXECUTOR_ID) val defaultName = MetricRegistry.name(source.sourceName) if (instance == "driver" || instance == "executor") { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 71940ee7d803e..193ebfffe6c3f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -717,7 +717,7 @@ private[spark] class TaskSetManager( calculatedTasks += 1 if (maxResultSize > 0 && totalResultSize > maxResultSize) { val msg = s"Total size of serialized results of ${calculatedTasks} tasks " + - s"(${Utils.bytesToString(totalResultSize)}) is bigger than spark.driver.maxResultSize " + + s"(${Utils.bytesToString(totalResultSize)}) is bigger than ${config.MAX_RESULT_SIZE.key} " + s"(${Utils.bytesToString(maxResultSize)})" logError(msg) abort(msg) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index aa4e63827bec4..31d66c833d15e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -18,13 +18,17 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future +import org.apache.hadoop.security.UserGroupInformation + import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ @@ -95,6 +99,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 + // Current set of delegation tokens to send to executors. + private val delegationTokens = new AtomicReference[Array[Byte]]() + + // The token manager used to create security tokens. + private var delegationTokenManager: Option[HadoopDelegationTokenManager] = None + private val reviveThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") @@ -152,6 +162,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } case UpdateDelegationTokens(newDelegationTokens) => + SparkHadoopUtil.get.addDelegationTokens(newDelegationTokens, conf) + delegationTokens.set(newDelegationTokens) executorDataMap.values.foreach { ed => ed.executorEndpoint.send(UpdateDelegationTokens(newDelegationTokens)) } @@ -230,7 +242,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), - fetchHadoopDelegationTokens()) + Option(delegationTokens.get())) context.reply(reply) } @@ -390,6 +402,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // TODO (prashant) send conf instead of properties driverEndpoint = createDriverEndpointRef(properties) + + if (UserGroupInformation.isSecurityEnabled()) { + delegationTokenManager = createTokenManager() + delegationTokenManager.foreach { dtm => + dtm.setDriverRef(driverEndpoint) + val creds = if (dtm.renewalEnabled) { + dtm.start().getCredentials() + } else { + val creds = UserGroupInformation.getCurrentUser().getCredentials() + dtm.obtainDelegationTokens(creds) + creds + } + delegationTokens.set(SparkHadoopUtil.get.serialize(creds)) + } + } } protected def createDriverEndpointRef( @@ -416,6 +443,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def stop() { reviveThread.shutdownNow() stopExecutors() + delegationTokenManager.foreach(_.stop()) try { if (driverEndpoint != null) { driverEndpoint.askSync[Boolean](StopDriver) @@ -684,7 +712,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp true } - protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } + /** + * Create the delegation token manager to be used for the application. This method is called + * once during the start of the scheduler backend (so after the object has already been + * fully constructed), only if security is enabled in the Hadoop configuration. + */ + protected def createTokenManager(): Option[HadoopDelegationTokenManager] = None // SPARK-27112: We need to ensure that there is ordering of lock acquisition // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 6df821f5b3fd8..5543f58d24e6d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -25,7 +25,7 @@ import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ @@ -54,7 +54,7 @@ private[spark] class StandaloneSchedulerBackend( private val registrationBarrier = new Semaphore(0) - private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCores = conf.get(config.CORES_MAX) private val totalExpectedCores = maxCores.getOrElse(0) override def start() { @@ -69,8 +69,8 @@ private[spark] class StandaloneSchedulerBackend( // The endpoint for executors to talk to us val driverUrl = RpcEndpointAddress( - sc.conf.get("spark.driver.host"), - sc.conf.get("spark.driver.port").toInt, + sc.conf.get(config.DRIVER_HOST_ADDRESS), + sc.conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString val args = Seq( "--driver-url", driverUrl, @@ -79,11 +79,11 @@ private[spark] class StandaloneSchedulerBackend( "--cores", "{{CORES}}", "--app-id", "{{APP_ID}}", "--worker-url", "{{WORKER_URL}}") - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS) .map(Utils.splitCommandString).getOrElse(Seq.empty) - val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") + val classPathEntries = sc.conf.get(config.EXECUTOR_CLASS_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) - val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") + val libraryPathEntries = sc.conf.get(config.EXECUTOR_LIBRARY_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) // When testing, expose the parent class path to the child. This is processed by @@ -102,7 +102,7 @@ private[spark] class StandaloneSchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val webUrl = sc.ui.map(_.webUrl).getOrElse("") - val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val coresPerExecutor = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) // If we're using dynamic allocation, set our initial executor limit to 0 for now. // ExecutorAllocationManager will send the real initial limit to the Master later. val initialExecutorLimit = diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 0de57fbd5600c..6ff8bf29b006a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.nio.ByteBuffer import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -116,7 +116,7 @@ private[spark] class LocalSchedulerBackend( * @param conf Spark configuration. */ def getUserClasspath(conf: SparkConf): Seq[URL] = { - val userClassPathStr = conf.getOption("spark.executor.extraClassPath") + val userClassPathStr = conf.get(config.EXECUTOR_CLASS_PATH) userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) } diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index f60dcfddfdc20..70564eeefda88 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} private[spark] class JavaSerializationStream( @@ -137,8 +138,8 @@ private[spark] class JavaSerializerInstance( */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - private var extraDebugInfo = conf.getBoolean("spark.serializer.extraDebugInfo", true) + private var counterReset = conf.get(SERIALIZER_OBJECT_STREAM_RESET) + private var extraDebugInfo = conf.get(SERIALIZER_EXTRA_DEBUG_INFO) protected def this() = this(new SparkConf()) // For deserialization only diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 424a3ed9bb5b0..966dfa29050bd 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -38,6 +38,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ @@ -57,33 +58,34 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") + private val bufferSizeKb = conf.get(KRYO_SERIALIZER_BUFFER_SIZE) if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + - s"2048 mb, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} mb.") + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_BUFFER_SIZE.key} must be less than " + + s"2048 MiB, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} MiB.") } private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt - val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt + val maxBufferSizeMb = conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE).toInt if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + - s"2048 mb, got: + $maxBufferSizeMb mb.") + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} must be less " + + s"than 2048 MiB, got: $maxBufferSizeMb MiB.") } private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt - private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) - private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) - private val userRegistrators = conf.get("spark.kryo.registrator", "") - .split(',').map(_.trim) + private val referenceTracking = conf.get(KRYO_REFERENCE_TRACKING) + private val registrationRequired = conf.get(KRYO_REGISTRATION_REQUIRED) + private val userRegistrators = conf.get(KRYO_USER_REGISTRATORS) + .map(_.trim) .filter(!_.isEmpty) - private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") - .split(',').map(_.trim) + private val classesToRegister = conf.get(KRYO_CLASSES_TO_REGISTER) + .map(_.trim) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema // whether to use unsafe based IO for serialization - private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) + private val useUnsafe = conf.get(KRYO_USE_UNSAFE) + private val usePool = conf.get(KRYO_USE_POOL) def newKryoOutput(): KryoOutput = if (useUnsafe) { @@ -348,7 +350,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole } catch { case e: KryoException if e.getMessage.startsWith("Buffer overflow") => throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + - "increase spark.kryoserializer.buffer.max value.", e) + s"increase ${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} value.", e) } finally { releaseKryo(kryo) } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index c4dd47d23f91a..7cc3d5c6301d5 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -27,6 +27,7 @@ import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 import org.apache.spark.storage._ @@ -44,10 +45,9 @@ private[spark] class AppStatusListener( kvstore: ElementTrackingStore, conf: SparkConf, live: Boolean, + appStatusSource: Option[AppStatusSource] = None, lastUpdateTime: Option[Long] = None) extends SparkListener with Logging { - import config._ - private var sparkVersion = SPARK_VERSION private var appInfo: v1.ApplicationInfo = null private var appSummary = new AppSummary(0, 0) @@ -266,6 +266,11 @@ private[spark] class AppStatusListener( private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = { liveExecutors.get(execId).foreach { exec => exec.isBlacklisted = blacklisted + if (blacklisted) { + appStatusSource.foreach(_.BLACKLISTED_EXECUTORS.inc()) + } else { + appStatusSource.foreach(_.UNBLACKLISTED_EXECUTORS.inc()) + } liveUpdate(exec, System.nanoTime()) } } @@ -368,11 +373,34 @@ private[spark] class AppStatusListener( } job.status = event.jobResult match { - case JobSucceeded => JobExecutionStatus.SUCCEEDED - case JobFailed(_) => JobExecutionStatus.FAILED + case JobSucceeded => + appStatusSource.foreach{_.SUCCEEDED_JOBS.inc()} + JobExecutionStatus.SUCCEEDED + case JobFailed(_) => + appStatusSource.foreach{_.FAILED_JOBS.inc()} + JobExecutionStatus.FAILED } job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None + + for { + source <- appStatusSource + submissionTime <- job.submissionTime + completionTime <- job.completionTime + } { + source.JOB_DURATION.value.set(completionTime.getTime() - submissionTime.getTime()) + } + + // update global app status counters + appStatusSource.foreach { source => + source.COMPLETED_STAGES.inc(job.completedStages.size) + source.FAILED_STAGES.inc(job.failedStages) + source.COMPLETED_TASKS.inc(job.completedTasks) + source.FAILED_TASKS.inc(job.failedTasks) + source.KILLED_TASKS.inc(job.killedTasks) + source.SKIPPED_TASKS.inc(job.skippedTasks) + source.SKIPPED_STAGES.inc(job.skippedStages.size) + } update(job, now, last = true) if (job.status == JobExecutionStatus.SUCCEEDED) { appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala new file mode 100644 index 0000000000000..f6a21578ff499 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status + +import java.util.concurrent.atomic.AtomicLong + +import AppStatusSource.getCounter +import com.codahale.metrics.{Counter, Gauge, MetricRegistry} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Status.APP_STATUS_METRICS_ENABLED +import org.apache.spark.metrics.source.Source + +private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] { + override def getValue: Long = value.get() +} + +private[spark] class AppStatusSource extends Source { + + override implicit val metricRegistry = new MetricRegistry() + + override val sourceName = "appStatus" + + val jobDuration = new JobDuration(new AtomicLong(0L)) + + // Duration of each job in milliseconds + val JOB_DURATION = metricRegistry + .register(MetricRegistry.name("jobDuration"), jobDuration) + + val FAILED_STAGES = getCounter("stages", "failedStages") + + val SKIPPED_STAGES = getCounter("stages", "skippedStages") + + val COMPLETED_STAGES = getCounter("stages", "completedStages") + + val SUCCEEDED_JOBS = getCounter("jobs", "succeededJobs") + + val FAILED_JOBS = getCounter("jobs", "failedJobs") + + val COMPLETED_TASKS = getCounter("tasks", "completedTasks") + + val FAILED_TASKS = getCounter("tasks", "failedTasks") + + val KILLED_TASKS = getCounter("tasks", "killedTasks") + + val SKIPPED_TASKS = getCounter("tasks", "skippedTasks") + + val BLACKLISTED_EXECUTORS = getCounter("tasks", "blackListedExecutors") + + val UNBLACKLISTED_EXECUTORS = getCounter("tasks", "unblackListedExecutors") +} + +private[spark] object AppStatusSource { + + def getCounter(prefix: String, name: String)(implicit metricRegistry: MetricRegistry): Counter = { + metricRegistry.counter(MetricRegistry.name(prefix, name)) + } + + def createSource(conf: SparkConf): Option[AppStatusSource] = { + Option(conf.get(APP_STATUS_METRICS_ENABLED)) + .filter(identity) + .map { _ => new AppStatusSource() } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 84716f8471bd8..46839aa83cc89 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -532,10 +532,11 @@ private[spark] object AppStatusStore { /** * Create an in-memory store for a live application. */ - def createLiveStore(conf: SparkConf): AppStatusStore = { + def createLiveStore( + conf: SparkConf, + appStatusSource: Option[AppStatusSource] = None): AppStatusStore = { val store = new ElementTrackingStore(new InMemoryStore(), conf) - val listener = new AppStatusListener(store, conf, true) + val listener = new AppStatusListener(store, conf, true, appStatusSource) new AppStatusStore(store, listener = Some(listener)) } - } diff --git a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala index 863b0967f765e..5ec7d90bfaaba 100644 --- a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala +++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.{HashMap, ListBuffer} import com.google.common.util.concurrent.MoreExecutors import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Status._ import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -45,8 +46,6 @@ import org.apache.spark.util.kvstore._ */ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) extends KVStore { - import config._ - private val triggers = new HashMap[Class[_], Seq[Trigger[_]]]() private val flushTriggers = new ListBuffer[() => Unit]() private val executor = if (conf.get(ASYNC_TRACKING_ENABLED)) { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index ad3d3bffafe27..bedf61aa2e157 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -61,7 +61,7 @@ private[spark] abstract class LiveEntity { private class LiveJob( val jobId: Int, name: String, - submissionTime: Option[Date], + val submissionTime: Option[Date], val stageIds: Seq[Int], jobGroup: Option[String], numTasks: Int) extends LiveEntity { diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 3c4ee4eb6bbb9..f0ae26e7a88e5 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -21,6 +21,7 @@ import java.util.{Timer, TimerTask} import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.StageData /** @@ -33,8 +34,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { // Carriage return private val CR = '\r' // Update period of progress bar, in milliseconds - private val updatePeriodMSec = - sc.getConf.getTimeAsMs("spark.ui.consoleProgress.update.interval", "200") + private val updatePeriodMSec = sc.getConf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) // Delay to show up a progress bar, in milliseconds private val firstDelayMSec = 500L diff --git a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala new file mode 100644 index 0000000000000..fc9b50f14a083 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui + +import java.util.{Enumeration, Map => JMap} +import javax.servlet._ +import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.config.UI._ + +/** + * A servlet filter that implements HTTP security features. The following actions are taken + * for every request: + * + * - perform access control of authenticated requests. + * - check request data for disallowed content (e.g. things that could be used to create XSS + * attacks). + * - set response headers to prevent certain kinds of attacks. + * + * Request parameters are sanitized so that HTML content is escaped, and disallowed content is + * removed. + */ +private class HttpSecurityFilter( + conf: SparkConf, + securityMgr: SecurityManager) extends Filter { + + override def destroy(): Unit = { } + + override def init(config: FilterConfig): Unit = { } + + override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { + val hreq = req.asInstanceOf[HttpServletRequest] + val hres = res.asInstanceOf[HttpServletResponse] + hres.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + + if (!securityMgr.checkUIViewPermissions(hreq.getRemoteUser())) { + hres.sendError(HttpServletResponse.SC_FORBIDDEN, + "User is not authorized to access this page.") + return + } + + // SPARK-10589 avoid frame-related click-jacking vulnerability, using X-Frame-Options + // (see http://tools.ietf.org/html/rfc7034). By default allow framing only from the + // same origin, but allow framing for a specific named URI. + // Example: spark.ui.allowFramingFrom = https://example.com/ + val xFrameOptionsValue = conf.getOption("spark.ui.allowFramingFrom") + .map { uri => s"ALLOW-FROM $uri" } + .getOrElse("SAMEORIGIN") + + hres.setHeader("X-Frame-Options", xFrameOptionsValue) + hres.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION)) + if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) { + hres.setHeader("X-Content-Type-Options", "nosniff") + } + if (hreq.getScheme() == "https") { + conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach( + hres.setHeader("Strict-Transport-Security", _)) + } + + chain.doFilter(new XssSafeRequest(hreq), res) + } + +} + +private class XssSafeRequest(req: HttpServletRequest) extends HttpServletRequestWrapper(req) { + + private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r + + private val parameterMap: Map[String, Array[String]] = { + super.getParameterMap().asScala.map { case (name, values) => + stripXSS(name) -> values.map(stripXSS) + }.toMap + } + + override def getParameterMap(): JMap[String, Array[String]] = parameterMap.asJava + + override def getParameterNames(): Enumeration[String] = { + parameterMap.keys.iterator.asJavaEnumeration + } + + override def getParameterValues(name: String): Array[String] = parameterMap.get(name).orNull + + override def getParameter(name: String): String = { + parameterMap.get(name).flatMap(_.headOption).orNull + } + + private def stripXSS(str: String): String = { + if (str != null) { + // Remove new lines and single quotes, followed by escaping HTML version 4.0 + StringEscapeUtils.escapeHtml4(NEWLINE_AND_SINGLE_QUOTE_REGEX.replaceAllIn(str, "")) + } else { + null + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 316af9b79d286..1651acdb4322f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.ui import java.net.{URI, URL} +import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} @@ -39,7 +40,7 @@ import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.Utils /** @@ -423,6 +424,16 @@ private[spark] object JettyUtils extends Logging { } } + def addFilter( + handler: ServletContextHandler, + filter: String, + params: Map[String, String]): Unit = { + val holder = new FilterHolder() + holder.setClassName(filter) + params.foreach { case (k, v) => holder.setInitParameter(k, v) } + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + private def createRedirectHttpsHandler(securePort: Int, scheme: String): ContextHandler = { val redirectHandler: ContextHandler = new ContextHandler redirectHandler.setContextPath("/") @@ -510,7 +521,7 @@ private[spark] case class ServerInfo( boundPort: Int, securePort: Option[Int], conf: SparkConf, - private val rootHandler: ContextHandlerCollection) { + private val rootHandler: ContextHandlerCollection) extends Logging { def addHandler(handler: ServletContextHandler): Unit = { handler.setVirtualHosts(JettyUtils.toVirtualHosts(JettyUtils.SPARK_CONNECTOR_NAME)) @@ -537,4 +548,33 @@ private[spark] case class ServerInfo( threadPool.asInstanceOf[LifeCycle].stop } } + + /** + * Add filters, if any, to the given ServletContextHandlers. Always adds a filter at the end + * of the chain to perform security-related functions. + */ + private def addFilters(handler: ServletContextHandler, securityMgr: SecurityManager): Unit = { + conf.get(UI_FILTERS).foreach { filter => + logInfo(s"Adding filter to ${handler.getContextPath()}: $filter") + val oldParams = conf.getOption(s"spark.$filter.params").toSeq + .flatMap(Utils.stringToSeq) + .flatMap { param => + val parts = param.split("=") + if (parts.length == 2) Some(parts(0) -> parts(1)) else None + } + .toMap + + val newParams = conf.getAllWithPrefix(s"spark.$filter.param.").toMap + + JettyUtils.addFilter(handler, filter, oldParams ++ newParams) + } + + // This filter must come after user-installed filters, since that's where authentication + // filters are installed. This means that custom filters will see the request before it's + // been validated by the security filter. + val securityFilter = new HttpSecurityFilter(conf, securityMgr) + val holder = new FilterHolder(securityFilter) + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index d315ef66e0dc0..7378801f7b4e2 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1._ @@ -50,7 +51,7 @@ private[spark] class SparkUI private ( with Logging with UIRoot { - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = sc.map(_.conf.get(UI_KILL_ENABLED)).getOrElse(false) var appId: String = _ @@ -151,12 +152,11 @@ private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) } private[spark] object SparkUI { - val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val DEFAULT_POOL_NAME = "default" def getUIPort(conf: SparkConf): Int = { - conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + conf.get(UI_PORT) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index d5a60f52cbb0f..2ddd78accde84 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -21,6 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.internal.config.UI._ import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils, WebUIPage} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { @@ -29,7 +30,7 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec private def init(): Unit = { val threadDumpEnabled = - parent.sc.isDefined && parent.conf.getBoolean("spark.ui.threadDumpsEnabled", true) + parent.sc.isDefined && parent.conf.get(UI_THREAD_DUMPS_ENABLED) attachPage(new ExecutorsPage(this, threadDumpEnabled)) if (threadDumpEnabled) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index ca3a13e8847cf..93128bb70da0d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -27,6 +27,7 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.TaskLocality import org.apache.spark.status._ import org.apache.spark.status.api.v1._ @@ -63,7 +64,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We // TODO: We should consider increasing the number of this parameter over time // if we find that it's okay. - private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) + private val MAX_TIMELINE_TASKS = parent.conf.get(UI_TIMELINE_TASKS_MAXIMUM) private def getLocalitySummaryString(localitySummary: Map[String, Long]): String = { val names = Map( diff --git a/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala index 4b6602b50aa1c..add1146c90840 100644 --- a/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala @@ -25,8 +25,12 @@ import org.apache.spark.SparkException * Contains basic command line parsing functionality and methods to parse some common Spark CLI * options. */ -private[spark] trait CommandLineUtils { +private[spark] trait CommandLineUtils extends CommandLineLoggingUtils { + def main(args: Array[String]): Unit +} + +private[spark] trait CommandLineLoggingUtils { // Exposed for testing private[spark] var exitFn: Int => Unit = (exitCode: Int) => System.exit(exitCode) @@ -41,6 +45,4 @@ private[spark] trait CommandLineUtils { printMessage("Run with --help for usage help or --verbose for debug output") exitFn(1) } - - def main(args: Array[String]): Unit } diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index e5cccf39f9455..902e48fed3916 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} private[spark] object RpcUtils { @@ -26,8 +27,8 @@ private[spark] object RpcUtils { * Retrieve a `RpcEndpointRef` which is located in the driver via its name. */ def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) + val driverHost: String = conf.get(config.DRIVER_HOST_ADDRESS.key, "localhost") + val driverPort: Int = conf.getInt(config.DRIVER_PORT.key, 7077) Utils.checkHost(driverHost) rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name) } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8f86b472b9373..7bd018719f66d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -61,6 +61,9 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -1500,16 +1503,12 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } - private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = - "spark.worker.ui.compressedLogFileLengthCacheSize" - private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null private def getCompressedLogFileLengthCache( sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { if (compressedLogFileLengthCache == null) { - val compressedLogFileLengthCacheSize = sparkConf.getInt( - UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, - DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + val compressedLogFileLengthCacheSize = sparkConf.get( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF) compressedLogFileLengthCache = CacheBuilder.newBuilder() .maximumSize(compressedLogFileLengthCacheSize) .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { @@ -2276,7 +2275,7 @@ private[spark] object Utils extends Logging { s"${e.getMessage}: Service$serviceString failed after " + s"$maxRetries retries (on a random free port)! " + s"Consider explicitly setting the appropriate binding address for " + - s"the service$serviceString (for example spark.driver.bindAddress " + + s"the service$serviceString (for example ${DRIVER_BIND_ADDRESS.key} " + s"for SparkDriver) to the correct binding address." } else { s"${e.getMessage}: Service$serviceString failed after " + @@ -2426,8 +2425,7 @@ private[spark] object Utils extends Logging { // Returns the groups to which the current user belongs. def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { - val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", - "org.apache.spark.security.ShellBasedGroupsMappingProvider") + val groupProviderClassName = sparkConf.get(USER_GROUPS_MAPPING) if (groupProviderClassName != "") { try { val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. @@ -2573,8 +2571,7 @@ private[spark] object Utils extends Logging { * has its own mechanism to distribute jars. */ def getUserJars(conf: SparkConf): Seq[String] = { - val sparkJars = conf.getOption("spark.jars") - sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + conf.get(JARS).filter(_.nonEmpty) } /** diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 2f9ad4c8cc3e1..3188e0bd2b70d 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{IntParam, Utils} /** @@ -115,11 +115,9 @@ private[spark] object FileAppender extends Logging { /** Create the right appender based on Spark configuration */ def apply(inputStream: InputStream, file: File, conf: SparkConf): FileAppender = { - import RollingFileAppender._ - - val rollingStrategy = conf.get(STRATEGY_PROPERTY, STRATEGY_DEFAULT) - val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) - val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) + val rollingStrategy = conf.get(config.EXECUTOR_LOGS_ROLLING_STRATEGY) + val rollingSizeBytes = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE) + val rollingInterval = conf.get(config.EXECUTOR_LOGS_ROLLING_TIME_INTERVAL) def createTimeBasedAppender(): FileAppender = { val validatedParams: Option[(Long, String)] = rollingInterval match { diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 5d8cec8447b53..59439b68792e5 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -24,6 +24,7 @@ import com.google.common.io.Files import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf +import org.apache.spark.internal.config /** * Continuously appends data from input stream into the given file, and rolls @@ -44,10 +45,8 @@ private[spark] class RollingFileAppender( bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE ) extends FileAppender(inputStream, activeFile, bufferSize) { - import RollingFileAppender._ - - private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) - private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) + private val maxRetainedFiles = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES) + private val enableCompression = conf.get(config.EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION) /** Stop the appender */ override def stop() { @@ -82,7 +81,7 @@ private[spark] class RollingFileAppender( // Roll the log file and compress if enableCompression is true. private def rotateFile(activeFile: File, rolloverFile: File): Unit = { if (enableCompression) { - val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + val gzFile = new File(rolloverFile.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX) var gzOutputStream: GZIPOutputStream = null var inputStream: InputStream = null try { @@ -103,7 +102,7 @@ private[spark] class RollingFileAppender( // Check if the rollover file already exists. private def rolloverFileExist(file: File): Boolean = { - file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + file.exists || new File(file.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX).exists } /** Move the active log file to a new rollover file */ @@ -164,15 +163,7 @@ private[spark] class RollingFileAppender( * names of configurations that configure rolling file appenders. */ private[spark] object RollingFileAppender { - val STRATEGY_PROPERTY = "spark.executor.logs.rolling.strategy" - val STRATEGY_DEFAULT = "" - val INTERVAL_PROPERTY = "spark.executor.logs.rolling.time.interval" - val INTERVAL_DEFAULT = "daily" - val SIZE_PROPERTY = "spark.executor.logs.rolling.maxSize" - val SIZE_DEFAULT = (1024 * 1024).toString - val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 - val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" val GZIP_LOG_SUFFIX = ".gz" diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 48408ccc8f81b..4d0ff6f1541c6 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.google.common.io.ByteStreams import org.apache.hadoop.fs.Path +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} @@ -590,7 +591,7 @@ class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { try { val conf = new SparkConf() .set("spark.checkpoint.compress", "true") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext("local", "test", conf) sc.setCheckpointDir(checkpointDir.toString) val rdd = sc.makeRDD(1 to 20, numSlices = 1) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index a69045f119d3a..20e8f34c9d7c6 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -155,7 +155,7 @@ class ExecutorAllocationManagerSuite .set("spark.dynamicAllocation.maxExecutors", "15") .set("spark.dynamicAllocation.minExecutors", "3") .set("spark.dynamicAllocation.executorAllocationRatio", divisor.toString) - .set("spark.executor.cores", cores.toString) + .set(config.EXECUTOR_CORES, cores) val sc = new SparkContext(conf) contexts += sc var manager = sc.executorAllocationManager.get diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 61da4138896cd..f8adaf59fa0ae 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.util.ThreadUtils @@ -256,7 +257,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft .set("spark.task.reaper.enabled", "true") .set("spark.task.reaper.killTimeout", "-1") .set("spark.task.reaper.PollingInterval", "1s") - .set("spark.deploy.maxExecutorRetries", "1") + .set(MAX_EXECUTOR_RETRIES, 1) sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) // Add a listener to release the semaphore once any tasks are launched. diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e357299770a2e..9f0d2ac82ead2 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,11 +19,14 @@ package org.apache.spark import java.io.File import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files import java.security.PrivilegedExceptionAction +import java.util.Base64 import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} @@ -41,11 +44,11 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls", "user1,user2") - val securityManager = new SecurityManager(conf); + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + val securityManager = new SecurityManager(conf) assert(securityManager.isAuthenticationEnabled() === true) assert(securityManager.aclsEnabled() === true) assert(securityManager.checkUIViewPermissions("user1") === true) @@ -55,10 +58,10 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf for groups") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls.groups", "group1,group2") + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // default ShellBasedGroupsMappingProvider is used to resolve user groups val securityManager = new SecurityManager(conf); // assuming executing user does not belong to group1,group2 @@ -66,27 +69,27 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf2.set("spark.authenticate", "true") - conf2.set("spark.authenticate.secret", "good") - conf2.set("spark.ui.acls.enable", "true") - conf2.set("spark.ui.view.acls.groups", "group1,group2") + conf2.set(NETWORK_AUTH_ENABLED, true) + conf2.set(AUTH_SECRET, "good") + conf2.set(ACLS_ENABLE, true) + conf2.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // explicitly specify a custom GroupsMappingServiceProvider - conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf2.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager2 = new SecurityManager(conf2); + val securityManager2 = new SecurityManager(conf2) // group4,group5 do not match assert(securityManager2.checkUIViewPermissions("user1") === true) assert(securityManager2.checkUIViewPermissions("user2") === true) val conf3 = new SparkConf - conf3.set("spark.authenticate", "true") - conf3.set("spark.authenticate.secret", "good") - conf3.set("spark.ui.acls.enable", "true") - conf3.set("spark.ui.view.acls.groups", "group4,group5") + conf3.set(NETWORK_AUTH_ENABLED, true) + conf3.set(AUTH_SECRET, "good") + conf3.set(ACLS_ENABLE, true) + conf3.set(UI_VIEW_ACLS_GROUPS, Seq("group4", "group5")) // explicitly specify a bogus GroupsMappingServiceProvider - conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + conf3.set(USER_GROUPS_MAPPING, "BogusServiceProvider") - val securityManager3 = new SecurityManager(conf3); + val securityManager3 = new SecurityManager(conf3) // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set assert(securityManager3.checkUIViewPermissions("user1") === false) assert(securityManager3.checkUIViewPermissions("user2") === false) @@ -94,7 +97,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with api") { val conf = new SparkConf - conf.set("spark.ui.view.acls", "user1,user2") + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -106,7 +109,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) - securityManager.setViewAcls(Set[String]("user5"), "user6,user7") + securityManager.setViewAcls(Set[String]("user5"), Seq("user6", "user7")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) @@ -117,41 +120,41 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with api for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setViewAclsGroups("group1,group2") + securityManager.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 match assert(securityManager.checkUIViewPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user2") === true) // change groups so they do not match - securityManager.setViewAclsGroups("group4,group5") + securityManager.setViewAclsGroups(Seq("group4", "group5")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf.set("spark.user.groups.mapping", "BogusServiceProvider") + conf.set(USER_GROUPS_MAPPING, "BogusServiceProvider") val securityManager2 = new SecurityManager(conf2) securityManager2.setAcls(true) - securityManager2.setViewAclsGroups("group1,group2") + securityManager2.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 do not match because of BogusServiceProvider assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) // setting viewAclsGroups to empty should still not match because of BogusServiceProvider - securityManager2.setViewAclsGroups("") + securityManager2.setViewAclsGroups(Nil) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) } test("set security modify acls") { val conf = new SparkConf - conf.set("spark.modify.acls", "user1,user2") + conf.set(MODIFY_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) @@ -164,7 +167,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) - securityManager.setModifyAcls(Set("user5"), "user6,user7") + securityManager.setModifyAcls(Set("user5"), Seq("user6", "user7")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user5") === true) assert(securityManager.checkModifyPermissions("user6") === true) @@ -175,34 +178,35 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security modify acls for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setModifyAclsGroups("group1,group2") + securityManager.setModifyAclsGroups(Seq("group1", "group2")) // group1,group2 match assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) // change groups so they do not match - securityManager.setModifyAclsGroups("group4,group5") + securityManager.setModifyAclsGroups(Seq("group4", "group5")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user2") === false) // change so they match again - securityManager.setModifyAclsGroups("group2,group3") + securityManager.setModifyAclsGroups(Seq("group2", "group3")) + assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) } test("set security admin acls") { val conf = new SparkConf - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "user3") - conf.set("spark.modify.acls", "user4") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("user3")) + conf.set(MODIFY_ACLS, Seq("user4")) - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -219,9 +223,9 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user5") === false) assert(securityManager.checkUIViewPermissions(null) === true) - securityManager.setAdminAcls("user6") - securityManager.setViewAcls(Set[String]("user8"), "user9") - securityManager.setModifyAcls(Set("user11"), "user9") + securityManager.setAdminAcls(Seq("user6")) + securityManager.setViewAcls(Set[String]("user8"), Seq("user9")) + securityManager.setModifyAcls(Set("user11"), Seq("user9")) assert(securityManager.checkModifyPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user11") === true) assert(securityManager.checkModifyPermissions("user9") === true) @@ -238,12 +242,12 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security admin acls for groups") { val conf = new SparkConf - conf.set("spark.admin.acls.groups", "group1") - conf.set("spark.ui.view.acls.groups", "group2") - conf.set("spark.modify.acls.groups", "group3") - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(ADMIN_ACLS_GROUPS, Seq("group1")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group2")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group3")) + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -252,38 +256,38 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user1") === true) // change admin groups so they do not match. view and modify groups are set to admin groups - securityManager.setAdminAclsGroups("group4,group5") + securityManager.setAdminAclsGroups(Seq("group4", "group5")) // invoke the set ui and modify to propagate the changes - securityManager.setViewAclsGroups("") - securityManager.setModifyAclsGroups("") + securityManager.setViewAclsGroups(Nil) + securityManager.setModifyAclsGroups(Nil) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === false) // change modify groups so they match - securityManager.setModifyAclsGroups("group3") + securityManager.setModifyAclsGroups(Seq("group3")) assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user1") === false) // change view groups so they match - securityManager.setViewAclsGroups("group2") - securityManager.setModifyAclsGroups("group4") + securityManager.setViewAclsGroups(Seq("group2")) + securityManager.setModifyAclsGroups(Seq("group4")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === true) // change modify and view groups so they do not match - securityManager.setViewAclsGroups("group7") - securityManager.setModifyAclsGroups("group8") + securityManager.setViewAclsGroups(Seq("group7")) + securityManager.setModifyAclsGroups(Seq("group8")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "*") - conf.set("spark.modify.acls", "user4") + conf.set(ACLS_ENABLE.key, "true") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("*")) + conf.set(MODIFY_ACLS, Seq("user4")) val securityManager = new SecurityManager(conf) assert(securityManager.aclsEnabled() === true) @@ -297,22 +301,22 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user8") === false) // check for modifyAcls with * - securityManager.setModifyAcls(Set("user4"), "*") + securityManager.setModifyAcls(Set("user4"), Seq("*")) assert(securityManager.checkModifyPermissions("user7") === true) assert(securityManager.checkModifyPermissions("user8") === true) - securityManager.setAdminAcls("user1,user2") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") + securityManager.setAdminAcls(Seq("user1", "user2")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) assert(securityManager.checkUIViewPermissions("user5") === false) assert(securityManager.checkUIViewPermissions("user6") === false) assert(securityManager.checkModifyPermissions("user7") === false) assert(securityManager.checkModifyPermissions("user8") === false) // check for adminAcls with * - securityManager.setAdminAcls("user1,*") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") + securityManager.setAdminAcls(Seq("user1", "*")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user7") === true) @@ -321,10 +325,10 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with * in acls for groups") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls.groups", "group4,group5") - conf.set("spark.ui.view.acls.groups", "*") - conf.set("spark.modify.acls.groups", "group6") + conf.set(ACLS_ENABLE, true) + conf.set(ADMIN_ACLS_GROUPS, Seq("group4", "group5")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("*")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group6")) val securityManager = new SecurityManager(conf) assert(securityManager.aclsEnabled() === true) @@ -336,17 +340,17 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user2") === false) // check for modifyAcls with * - securityManager.setModifyAclsGroups("*") - securityManager.setViewAclsGroups("group6") + securityManager.setModifyAclsGroups(Seq("*")) + securityManager.setViewAclsGroups(Seq("group6")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) // check for adminAcls with * - securityManager.setAdminAclsGroups("group9,*") - securityManager.setModifyAclsGroups("group4,group5") - securityManager.setViewAclsGroups("group6,group7") + securityManager.setAdminAclsGroups(Seq("group9", "*")) + securityManager.setModifyAclsGroups(Seq("group4", "group5")) + securityManager.setViewAclsGroups(Seq("group6", "group7")) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user7") === true) @@ -365,13 +369,13 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user1") === false) // set groups only - securityManager.setAdminAclsGroups("group1,group2") + securityManager.setAdminAclsGroups(Seq("group1", "group2")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user1") === false) } test("missing secret authentication key") { - val conf = new SparkConf().set("spark.authenticate", "true") + val conf = new SparkConf().set(NETWORK_AUTH_ENABLED, true) val mgr = new SecurityManager(conf) intercept[IllegalArgumentException] { mgr.getSecretKey() @@ -395,15 +399,69 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(keyFromEnv === new SecurityManager(conf2).getSecretKey()) } - test("secret key generation") { - Seq( - ("yarn", true), - ("local", true), - ("local[*]", true), - ("local[1, 2]", true), - ("local-cluster[2, 1, 1024]", false), - ("invalid", false) - ).foreach { case (master, shouldGenerateSecret) => + test("use executor-specific secret file configuration.") { + val secretFileFromDriver = createTempSecretFile("driver-secret") + val secretFileFromExecutor = createTempSecretFile("executor-secret") + val conf = new SparkConf() + .setMaster("k8s://127.0.0.1") + .set(AUTH_SECRET_FILE_DRIVER, Some(secretFileFromDriver.getAbsolutePath)) + .set(AUTH_SECRET_FILE_EXECUTOR, Some(secretFileFromExecutor.getAbsolutePath)) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr = new SecurityManager(conf, authSecretFileConf = AUTH_SECRET_FILE_EXECUTOR) + assert(encodeFileAsBase64(secretFileFromExecutor) === mgr.getSecretKey()) + } + + test("secret file must be defined in both driver and executor") { + val conf1 = new SparkConf() + .set(AUTH_SECRET_FILE_DRIVER, Some("/tmp/driver-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr1 = new SecurityManager(conf1) + intercept[IllegalArgumentException] { + mgr1.initializeAuth() + } + + val conf2 = new SparkConf() + .set(AUTH_SECRET_FILE_EXECUTOR, Some("/tmp/executor-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr2 = new SecurityManager(conf2) + intercept[IllegalArgumentException] { + mgr2.initializeAuth() + } + } + + Seq("yarn", "local", "local[*]", "local[1,2]", "mesos://localhost:8080").foreach { master => + test(s"master $master cannot use file mounted secrets") { + val conf = new SparkConf() + .set(AUTH_SECRET_FILE, "/tmp/secret.txt") + .set(SecurityManager.SPARK_AUTH_CONF, "true") + .setMaster(master) + intercept[IllegalArgumentException] { + new SecurityManager(conf).getSecretKey() + } + intercept[IllegalArgumentException] { + new SecurityManager(conf).initializeAuth() + } + } + } + + // How is the secret expected to be generated and stored. + object SecretTestType extends Enumeration { + val MANUAL, AUTO, UGI, FILE = Value + } + + import SecretTestType._ + + Seq( + ("yarn", UGI), + ("local", UGI), + ("local[*]", UGI), + ("local[1, 2]", UGI), + ("k8s://127.0.0.1", AUTO), + ("k8s://127.0.1.1", FILE), + ("local-cluster[2, 1, 1024]", MANUAL), + ("invalid", MANUAL) + ).foreach { case (master, secretType) => + test(s"secret key generation: master '$master'") { val conf = new SparkConf() .set(NETWORK_AUTH_ENABLED, true) .set(SparkLauncher.SPARK_MASTER, master) @@ -412,19 +470,32 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { UserGroupInformation.createUserForTesting("authTest", Array()).doAs( new PrivilegedExceptionAction[Unit]() { override def run(): Unit = { - if (shouldGenerateSecret) { - mgr.initializeAuth() - val creds = UserGroupInformation.getCurrentUser().getCredentials() - val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) - assert(secret != null) - assert(new String(secret, UTF_8) === mgr.getSecretKey()) - } else { - intercept[IllegalArgumentException] { + secretType match { + case UGI => + mgr.initializeAuth() + val creds = UserGroupInformation.getCurrentUser().getCredentials() + val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) + assert(secret != null) + assert(new String(secret, UTF_8) === mgr.getSecretKey()) + + case AUTO => mgr.initializeAuth() - } - intercept[IllegalArgumentException] { - mgr.getSecretKey() - } + val creds = UserGroupInformation.getCurrentUser().getCredentials() + assert(creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) === null) + + case MANUAL => + intercept[IllegalArgumentException] { + mgr.initializeAuth() + } + intercept[IllegalArgumentException] { + mgr.getSecretKey() + } + + case FILE => + val secretFile = createTempSecretFile() + conf.set(AUTH_SECRET_FILE, secretFile.getAbsolutePath) + mgr.initializeAuth() + assert(encodeFileAsBase64(secretFile) === mgr.getSecretKey()) } } } @@ -432,5 +503,15 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { } } + private def encodeFileAsBase64(secretFile: File) = { + Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + } + + private def createTempSecretFile(contents: String = "test-secret"): File = { + val secretDir = Utils.createTempDir("temp-secrets") + val secretFile = new File(secretDir, "temp-secret.txt") + Files.write(secretFile.toPath, contents.getBytes(UTF_8)) + secretFile + } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b917469e48747..26b9a6360c3e8 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,8 @@ import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.internal.config.SERIALIZER +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} @@ -214,7 +216,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC test("sort with Java non serializable class - Kryo") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - val myConf = conf.clone().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + val myConf = conf.clone().set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") sc = new SparkContext("local-cluster[2,1,1024]", "test", myConf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 0d06b02e74e34..379335555446a 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -26,8 +26,9 @@ import scala.util.{Random, Try} import com.esotericsoftware.kryo.Kryo -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, KryoSerializer} import org.apache.spark.util.{ResetSystemProperties, RpcUtils} @@ -78,7 +79,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(conf.get("spark.master") === "local[3]") assert(conf.get("spark.app.name") === "My app") assert(conf.get("spark.home") === "/path") - assert(conf.get("spark.jars") === "a.jar,b.jar") + assert(conf.get(JARS) === Seq("a.jar", "b.jar")) assert(conf.get("spark.executorEnv.VAR1") === "value1") assert(conf.get("spark.executorEnv.VAR2") === "value2") assert(conf.get("spark.executorEnv.VAR3") === "value3") @@ -86,7 +87,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // Test the Java-friendly versions of these too conf.setJars(Array("c.jar", "d.jar")) conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5"))) - assert(conf.get("spark.jars") === "c.jar,d.jar") + assert(conf.get(JARS) === Seq("c.jar", "d.jar")) assert(conf.get("spark.executorEnv.VAR4") === "value4") assert(conf.get("spark.executorEnv.VAR5") === "value5") } @@ -138,6 +139,13 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(sc.appName === "My other app") } + test("creating SparkContext with cpus per tasks bigger than cores per executors") { + val conf = new SparkConf(false) + .set(EXECUTOR_CORES, 1) + .set("spark.task.cpus", "2") + intercept[SparkException] { sc = new SparkContext(conf) } + } + test("nested property names") { // This wasn't supported by some external conf parsing libraries System.setProperty("spark.test.a", "a") @@ -175,19 +183,19 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1], classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName).toSet) conf.registerKryoClasses(Array(classOf[Class3])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) conf.registerKryoClasses(Array(classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -198,12 +206,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses and custom registrator") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1])) - assert(conf.get("spark.kryo.classesToRegister") === classOf[Class1].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === Seq(classOf[Class1].getName).toSet) - conf.set("spark.kryo.registrator", classOf[CustomRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[CustomRegistrator].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -213,9 +221,9 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through conf") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") - conf.set("spark.kryo.classesToRegister", "java.lang.StringBuffer") - conf.set("spark.serializer", classOf[KryoSerializer].getName) + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) + conf.set(KRYO_CLASSES_TO_REGISTER, Seq("java.lang.StringBuffer")) + conf.set(SERIALIZER, classOf[KryoSerializer].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index e1666a35271d3..a0a052645aa39 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -34,6 +34,10 @@ import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index b38a3667abee1..7390c2ebc2558 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -24,6 +24,7 @@ import scala.io.Source import org.scalatest.Matchers import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -49,7 +50,7 @@ class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkC } val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) assertBroadcastIsValid(broadcast) - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) val deserializedBroadcast = Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) assertBroadcastIsValid(deserializedBroadcast) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/util/Benchmark.scala rename to core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 7def44bd2a2b1..86e732db22057 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.benchmark import java.io.{OutputStream, PrintStream} @@ -27,6 +27,8 @@ import scala.util.Try import org.apache.commons.io.output.TeeOutputStream import org.apache.commons.lang3.SystemUtils +import org.apache.spark.util.Utils + /** * Utility class to benchmark components. An example of how to use this is: * val benchmark = new Benchmark("My Benchmark", valuesPerIteration) @@ -46,13 +48,13 @@ import org.apache.commons.lang3.SystemUtils * @param output optional output stream to write benchmark results to */ private[spark] class Benchmark( - name: String, - valuesPerIteration: Long, - minNumIters: Int = 2, - warmupTime: FiniteDuration = 2.seconds, - minTime: FiniteDuration = 2.seconds, - outputPerIteration: Boolean = false, - output: Option[OutputStream] = None) { + name: String, + valuesPerIteration: Long, + minNumIters: Int = 2, + warmupTime: FiniteDuration = 2.seconds, + minTime: FiniteDuration = 2.seconds, + outputPerIteration: Boolean = false, + output: Option[OutputStream] = None) { import Benchmark._ val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] @@ -63,12 +65,12 @@ private[spark] class Benchmark( } /** - * Adds a case to run when run() is called. The given function will be run for several - * iterations to collect timing statistics. - * - * @param name of the benchmark case - * @param numIters if non-zero, forces exactly this many iterations to be run - */ + * Adds a case to run when run() is called. The given function will be run for several + * iterations to collect timing statistics. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run + */ def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { addTimerCase(name, numIters) { timer => timer.startTiming() @@ -78,22 +80,22 @@ private[spark] class Benchmark( } /** - * Adds a case with manual timing control. When the function is run, timing does not start - * until timer.startTiming() is called within the given function. The corresponding - * timer.stopTiming() method must be called before the function returns. - * - * @param name of the benchmark case - * @param numIters if non-zero, forces exactly this many iterations to be run - */ + * Adds a case with manual timing control. When the function is run, timing does not start + * until timer.startTiming() is called within the given function. The corresponding + * timer.stopTiming() method must be called before the function returns. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run + */ def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { benchmarks += Benchmark.Case(name, f, numIters) } /** - * Runs the benchmark and outputs the results to stdout. This should be copied and added as - * a comment with the benchmark. Although the results vary from machine to machine, it should - * provide some baseline. - */ + * Runs the benchmark and outputs the results to stdout. This should be copied and added as + * a comment with the benchmark. Although the results vary from machine to machine, it should + * provide some baseline. + */ def run(): Unit = { require(benchmarks.nonEmpty) // scalastyle:off @@ -109,13 +111,15 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. out.println(Benchmark.getJVMOSInfo()) out.println(Benchmark.getProcessorName()) - out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", name + ":", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - out.println("-" * 96) + out.println("-" * 120) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %16s %12s %13s %10s\n", + out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", benchmark.name, - "%5.0f / %4.0f" format (result.bestMs, result.avgMs), + "%5.0f" format result.bestMs, + "%4.0f" format result.avgMs, + "%5.0f" format result.stdevMs, "%10.1f" format result.bestRate, "%6.1f" format (1000 / result.bestRate), "%3.1fX" format (firstBest / result.bestMs)) @@ -125,9 +129,9 @@ private[spark] class Benchmark( } /** - * Runs a single function `f` for iters, returning the average time the function took and - * the rate of the function. - */ + * Runs a single function `f` for iters, returning the average time the function took and + * the rate of the function. + */ def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = { System.gc() // ensures garbage from previous cases don't impact this one val warmupDeadline = warmupTime.fromNow @@ -146,27 +150,31 @@ private[spark] class Benchmark( if (outputPerIteration) { // scalastyle:off - println(s"Iteration $i took ${runTime / 1000} microseconds") + println(s"Iteration $i took ${NANOSECONDS.toMicros(runTime)} microseconds") // scalastyle:on } i += 1 } // scalastyle:off - println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms") + println(s" Stopped after $i iterations, ${NANOSECONDS.toMillis(runTimes.sum)} ms") // scalastyle:on + assert(runTimes.nonEmpty) val best = runTimes.min val avg = runTimes.sum / runTimes.size - Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) + val stdev = if (runTimes.size > 1) { + math.sqrt(runTimes.map(time => (time - avg) * (time - avg)).sum / (runTimes.size - 1)) + } else 0 + Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0, stdev / 1000000.0) } } private[spark] object Benchmark { /** - * Object available to benchmark code to control timing e.g. to exclude set-up time. - * - * @param iteration specifies this is the nth iteration of running the benchmark case - */ + * Object available to benchmark code to control timing e.g. to exclude set-up time. + * + * @param iteration specifies this is the nth iteration of running the benchmark case + */ class Timer(val iteration: Int) { private var accumulatedTime: Long = 0L private var timeStart: Long = 0L @@ -189,20 +197,21 @@ private[spark] object Benchmark { } case class Case(name: String, fn: Timer => Unit, numIters: Int) - case class Result(avgMs: Double, bestRate: Double, bestMs: Double) + case class Result(avgMs: Double, bestRate: Double, bestMs: Double, stdevMs: Double) /** - * This should return a user helpful processor information. Getting at this depends on the OS. - * This should return something like "Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz" - */ + * This should return a user helpful processor information. Getting at this depends on the OS. + * This should return something like "Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz" + */ def getProcessorName(): String = { val cpu = if (SystemUtils.IS_OS_MAC_OSX) { Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) + .stripLineEnd } else if (SystemUtils.IS_OS_LINUX) { Try { val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")).stripLineEnd Utils.executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) - .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") + .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") }.getOrElse("Unknown processor") } else { System.getenv("PROCESSOR_IDENTIFIER") @@ -211,10 +220,10 @@ private[spark] object Benchmark { } /** - * This should return a user helpful JVM & OS information. - * This should return something like - * "OpenJDK 64-Bit Server VM 1.8.0_65-b17 on Linux 4.1.13-100.fc21.x86_64" - */ + * This should return a user helpful JVM & OS information. + * This should return something like + * "OpenJDK 64-Bit Server VM 1.8.0_65-b17 on Linux 4.1.13-100.fc21.x86_64" + */ def getJVMOSInfo(): String = { val vmName = System.getProperty("java.vm.name") val runtimeVersion = System.getProperty("java.runtime.version") diff --git a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala new file mode 100644 index 0000000000000..a6666db4e95c3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.benchmark + +import java.io.{File, FileOutputStream, OutputStream} + +/** + * A base class for generate benchmark results to a file. + */ +abstract class BenchmarkBase { + var output: Option[OutputStream] = None + + /** + * Main process of the whole benchmark. + * Implementations of this method are supposed to use the wrapper method `runBenchmark` + * for each benchmark scenario. + */ + def runBenchmarkSuite(mainArgs: Array[String]): Unit + + final def runBenchmark(benchmarkName: String)(func: => Any): Unit = { + val separator = "=" * 96 + val testHeader = (separator + '\n' + benchmarkName + '\n' + separator + '\n' + '\n').getBytes + output.foreach(_.write(testHeader)) + func + output.foreach(_.write('\n')) + } + + def main(args: Array[String]): Unit = { + val regenerateBenchmarkFiles: Boolean = System.getenv("SPARK_GENERATE_BENCHMARK_FILES") == "1" + if (regenerateBenchmarkFiles) { + val resultFileName = s"${this.getClass.getSimpleName.replace("$", "")}-results.txt" + val file = new File(s"benchmarks/$resultFileName") + if (!file.exists()) { + file.createNewFile() + } + output = Some(new FileOutputStream(file)) + } + + runBenchmarkSuite(args) + + output.foreach { o => + if (o != null) { + o.close() + } + } + + afterAll() + } + + /** + * Any shutdown code to ensure a clean shutdown + */ + def afterAll(): Unit = {} +} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 9ad2e9a5e74ac..6ae789fdabefa 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -24,6 +24,8 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.internal.config.SERIALIZER import org.apache.spark.io.SnappyCompressionCodec import org.apache.spark.rdd.RDD import org.apache.spark.security.EncryptionFunSuite @@ -67,8 +69,8 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio encryptionTest("Accessing TorrentBroadcast variables in a local cluster") { conf => val numSlaves = 4 - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.broadcast.compress", "true") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(config.BROADCAST_COMPRESS, true) sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 887a88f567fe7..d8a4538dd91c7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -42,6 +42,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} @@ -220,7 +221,7 @@ class SparkSubmitSuite val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) appArgs.deployMode should be ("client") - conf.get("spark.submit.deployMode") should be ("client") + conf.get(SUBMIT_DEPLOY_MODE) should be ("client") // Both cmd line and configuration are specified, cmdline option takes the priority val clArgs1 = Seq( @@ -234,7 +235,7 @@ class SparkSubmitSuite val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1) appArgs1.deployMode should be ("cluster") - conf1.get("spark.submit.deployMode") should be ("cluster") + conf1.get(SUBMIT_DEPLOY_MODE) should be ("cluster") // Neither cmdline nor configuration are specified, client mode is the default choice val clArgs2 = Seq( @@ -247,7 +248,7 @@ class SparkSubmitSuite val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) appArgs2.deployMode should be ("client") - conf2.get("spark.submit.deployMode") should be ("client") + conf2.get(SUBMIT_DEPLOY_MODE) should be ("client") } test("handles YARN cluster mode") { @@ -289,7 +290,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.app.name") should be ("beauty") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -328,7 +329,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -373,13 +374,13 @@ class SparkSubmitSuite val confMap = conf.getAll.toMap confMap.keys should contain ("spark.master") confMap.keys should contain ("spark.app.name") - confMap.keys should contain ("spark.jars") + confMap.keys should contain (JARS.key) confMap.keys should contain ("spark.driver.memory") confMap.keys should contain ("spark.driver.cores") confMap.keys should contain ("spark.driver.supervise") - confMap.keys should contain ("spark.ui.enabled") - confMap.keys should contain ("spark.submit.deployMode") - conf.get("spark.ui.enabled") should be ("false") + confMap.keys should contain (UI_ENABLED.key) + confMap.keys should contain (SUBMIT_DEPLOY_MODE.key) + conf.get(UI_ENABLED) should be (false) } test("handles standalone client mode") { @@ -401,7 +402,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles mesos client mode") { @@ -423,7 +424,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles k8s cluster mode") { @@ -466,7 +467,7 @@ class SparkSubmitSuite val (_, _, conf, mainClass) = submit.prepareSubmitEnvironment(appArgs) conf.get("spark.executor.memory") should be ("5g") conf.get("spark.master") should be ("yarn") - conf.get("spark.submit.deployMode") should be ("cluster") + conf.get(SUBMIT_DEPLOY_MODE) should be ("cluster") mainClass should be (SparkSubmit.YARN_CLUSTER_SUBMIT_CLASS) } @@ -644,59 +645,60 @@ class SparkSubmitSuite } test("resolves command line argument paths correctly") { - val dir = Utils.createTempDir() - val archive = Paths.get(dir.toPath.toString, "single.zip") - Files.createFile(archive) - val jars = "/jar1,/jar2" - val files = "local:/file1,file2" - val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" - val pyFiles = "py-file1,py-file2" - - // Test jars and files - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--jars", jars, - "--files", files, - "thejar.jar") - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - appArgs.jars should be (Utils.resolveURIs(jars)) - appArgs.files should be (Utils.resolveURIs(files)) - conf.get("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be (Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--files", files, - "--archives", archives, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - appArgs2.files should be (Utils.resolveURIs(files)) - appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") - conf2.get("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should fullyMatch regex - ("file:/archive1,file:.*#archive3") - - // Test python files - val clArgs3 = Seq( - "--master", "local", - "--py-files", pyFiles, - "--conf", "spark.pyspark.driver.python=python3.4", - "--conf", "spark.pyspark.python=python3.5", - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) - conf3.get("spark.submit.pyFiles") should be ( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - conf3.get(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4") - conf3.get(PYSPARK_PYTHON.key) should be ("python3.5") +// withTempDir { dir => +// val archive = Paths.get(dir.toPath.toString, "single.zip") +// Files.createFile(archive) +// val jars = "/jar1,/jar2" +// val files = "local:/file1,file2" +// val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" +// val pyFiles = "py-file1,py-file2" +// +// // Test jars and files +// val clArgs = Seq( +// "--master", "local", +// "--class", "org.SomeClass", +// "--jars", jars, +// "--files", files, +// "thejar.jar") +// val appArgs = new SparkSubmitArguments(clArgs) +// val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) +// appArgs.jars should be(Utils.resolveURIs(jars)) +// appArgs.files should be(Utils.resolveURIs(files)) +// conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) +// conf.get("spark.files") should be(Utils.resolveURIs(files)) +// +// // Test files and archives (Yarn) +// val clArgs2 = Seq( +// "--master", "yarn", +// "--class", "org.SomeClass", +// "--files", files, +// "--archives", archives, +// "thejar.jar" +// ) +// val appArgs2 = new SparkSubmitArguments(clArgs2) +// val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) +// appArgs2.files should be(Utils.resolveURIs(files)) +// appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") +// conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) +// conf2.get("spark.yarn.dist.archives") should fullyMatch regex +// ("file:/archive1,file:.*#archive3") +// +// // Test python files +// val clArgs3 = Seq( +// "--master", "local", +// "--py-files", pyFiles, +// "--conf", "spark.pyspark.driver.python=python3.4", +// "--conf", "spark.pyspark.python=python3.5", +// "mister.py" +// ) +// val appArgs3 = new SparkSubmitArguments(clArgs3) +// val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) +// appArgs3.pyFiles should be(Utils.resolveURIs(pyFiles)) +// conf3.get(SUBMIT_PYTHON_FILES) should be( +// PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) +// conf3.get(PYSPARK_DRIVER_PYTHON.key) should be("python3.4") +// conf3.get(PYSPARK_PYTHON.key) should be("python3.5") +// } } test("ambiguous archive mapping results in error message") { @@ -728,77 +730,77 @@ class SparkSubmitSuite val archives = "file:/archive1,archive2" // spark.yarn.dist.archives val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles - val tmpDir = Utils.createTempDir() - - // Test jars and files - val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) - val writer1 = new PrintWriter(f1) - writer1.println("spark.jars " + jars) - writer1.println("spark.files " + files) - writer1.close() - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--properties-file", f1.getPath, - "thejar.jar" - ) - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be(Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) - val writer2 = new PrintWriter(f2) - writer2.println("spark.yarn.dist.files " + files) - writer2.println("spark.yarn.dist.archives " + archives) - writer2.close() - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--properties-file", f2.getPath, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) - - // Test python files - val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) - val writer3 = new PrintWriter(f3) - writer3.println("spark.submit.pyFiles " + pyFiles) - writer3.close() - val clArgs3 = Seq( - "--master", "local", - "--properties-file", f3.getPath, - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - conf3.get("spark.submit.pyFiles") should be( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - - // Test remote python files - val hadoopConf = new Configuration() - updateConfWithFakeS3Fs(hadoopConf) - val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) - val pyFile1 = File.createTempFile("file1", ".py", tmpDir) - val pyFile2 = File.createTempFile("file2", ".py", tmpDir) - val writer4 = new PrintWriter(f4) - val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" - writer4.println("spark.submit.pyFiles " + remotePyFiles) - writer4.close() - val clArgs4 = Seq( - "--master", "yarn", - "--deploy-mode", "cluster", - "--properties-file", f4.getPath, - "hdfs:///tmp/mister.py" - ) - val appArgs4 = new SparkSubmitArguments(clArgs4) - val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) - // Should not format python path for yarn cluster mode - conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles)) +// withTempDir { tmpDir => +// // Test jars and files +// val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) +// val writer1 = new PrintWriter(f1) +// writer1.println("spark.jars " + jars) +// writer1.println("spark.files " + files) +// writer1.close() +// val clArgs = Seq( +// "--master", "local", +// "--class", "org.SomeClass", +// "--properties-file", f1.getPath, +// "thejar.jar" +// ) +// val appArgs = new SparkSubmitArguments(clArgs) +// val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) +// conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) +// conf.get(FILES) should be(Utils.resolveURIs(files).split(",").toSeq) +// +// // Test files and archives (Yarn) +// val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) +// val writer2 = new PrintWriter(f2) +// writer2.println("spark.yarn.dist.files " + files) +// writer2.println("spark.yarn.dist.archives " + archives) +// writer2.close() +// val clArgs2 = Seq( +// "--master", "yarn", +// "--class", "org.SomeClass", +// "--properties-file", f2.getPath, +// "thejar.jar" +// ) +// val appArgs2 = new SparkSubmitArguments(clArgs2) +// val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) +// conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) +// conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) +// +// // Test python files +// val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) +// val writer3 = new PrintWriter(f3) +// writer3.println("spark.submit.pyFiles " + pyFiles) +// writer3.close() +// val clArgs3 = Seq( +// "--master", "local", +// "--properties-file", f3.getPath, +// "mister.py" +// ) +// val appArgs3 = new SparkSubmitArguments(clArgs3) +// val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) +// conf3.get(SUBMIT_PYTHON_FILES) should be( +// PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) +// +// // Test remote python files +// val hadoopConf = new Configuration() +// updateConfWithFakeS3Fs(hadoopConf) +// val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) +// val pyFile1 = File.createTempFile("file1", ".py", tmpDir) +// val pyFile2 = File.createTempFile("file2", ".py", tmpDir) +// val writer4 = new PrintWriter(f4) +// val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" +// writer4.println("spark.submit.pyFiles " + remotePyFiles) +// writer4.close() +// val clArgs4 = Seq( +// "--master", "yarn", +// "--deploy-mode", "cluster", +// "--properties-file", f4.getPath, +// "hdfs:///tmp/mister.py" +// ) +// val appArgs4 = new SparkSubmitArguments(clArgs4) +// val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) +// // Should not format python path for yarn cluster mode +// conf4.get(SUBMIT_PYTHON_FILES) should be(Utils.resolveURIs(remotePyFiles).split(",")) +// } } test("user classpath first in driver") { @@ -1019,7 +1021,7 @@ class SparkSubmitSuite conf.get("spark.repl.local.jars") should (startWith("file:")) // local py files should not be a URI format. - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } } test("download remote resource if it is not supported by yarn service") { @@ -1153,8 +1155,8 @@ class SparkSubmitSuite val appArgs = new SparkSubmitArguments(args) val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) - conf.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } // Verify "spark.submit.pyFiles" val args1 = Seq( @@ -1169,8 +1171,8 @@ class SparkSubmitSuite val appArgs1 = new SparkSubmitArguments(args1) val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) - conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf1.get("spark.submit.pyFiles") should (startWith("/")) + conf1.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } } test("handles natural line delimiters in --properties-file and --conf uniformly") { @@ -1219,6 +1221,23 @@ class SparkSubmitSuite conf.get(nonDelimSpaceFromFile._1) should be ("blah") } + + test("get a Spark configuration from arguments") { + val testConf = "spark.test.hello" -> "world" + val masterConf = "spark.master" -> "yarn" + val clArgs = Seq( + "--conf", s"${testConf._1}=${testConf._2}", + "--conf", s"${masterConf._1}=${masterConf._2}", + "--class", "Foo", + "app.jar") + val conf = new SparkSubmitArguments(clArgs).toSparkConf() + Seq( + testConf, + masterConf + ).foreach { case (k, v) => + conf.get(k) should be (v) + } + } } object SparkSubmitSuite extends SparkFunSuite with TimeLimits { diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index a1d2a1283db14..8567dd1f08233 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -243,7 +243,7 @@ class StandaloneDynamicAllocationSuite } test("dynamic allocation with cores per executor") { - sc = new SparkContext(appConf.set("spark.executor.cores", "2")) + sc = new SparkContext(appConf.set(config.EXECUTOR_CORES, 2)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -296,7 +296,7 @@ class StandaloneDynamicAllocationSuite test("dynamic allocation with cores per executor AND max cores") { sc = new SparkContext(appConf - .set("spark.executor.cores", "2") + .set(config.EXECUTOR_CORES, 2) .set("spark.cores.max", "8")) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { @@ -526,7 +526,7 @@ class StandaloneDynamicAllocationSuite new SparkConf() .setMaster(masterRpcEnv.address.toSparkURL) .setAppName("test") - .set("spark.executor.memory", "256m") + .set(config.EXECUTOR_MEMORY.key, "256m") } /** Make a master to which our application will send executor requests. */ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 98ffd722b6f98..225b4f14312bd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -39,8 +39,9 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ import org.apache.spark.scheduler._ import org.apache.spark.security.GroupMappingServiceProvider @@ -584,12 +585,12 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test both history ui admin acls and application acls are configured. val conf1 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) - createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) { + createAndCheck(conf1, (ADMIN_ACLS.key, "user"), (ADMIN_ACLS_GROUPS.key, "group")) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -606,10 +607,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test only history ui admin acls are configured. val conf2 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf2) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -626,8 +627,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test neither history ui admin acls nor application acls are configured. val conf3 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf3) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (false) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala index 4b1b921582e00..341a1e2443df0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala @@ -25,14 +25,13 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.History._ import org.apache.spark.status.KVUtils import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.util.kvstore.KVStore class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { - import config._ - private val MAX_USAGE = 3L private var testDir: File = _ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 11b29121739a4..f5438e44836cc 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -45,7 +45,9 @@ import org.scalatest.mockito.MockitoSugar import org.scalatest.selenium.WebBrowser import org.apache.spark._ -import org.apache.spark.deploy.history.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.ApplicationInfo import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI @@ -606,9 +608,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers stop() init( - "spark.ui.filters" -> classOf[FakeAuthFilter].getName(), - "spark.history.ui.acls.enable" -> "true", - "spark.history.ui.admin.acls" -> admin) + UI_FILTERS.key -> classOf[FakeAuthFilter].getName(), + HISTORY_SERVER_UI_ACLS_ENABLE.key -> "true", + HISTORY_SERVER_UI_ADMIN_ACLS.key -> admin) val tests = Seq( (owner, HttpServletResponse.SC_OK), diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 07830fdd3b4ce..fbf2acc3175d4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -39,6 +39,10 @@ import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy._ import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer @@ -101,10 +105,9 @@ class MasterSuite extends SparkFunSuite test("can use a custom recovery mode factory") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[CustomRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[CustomRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts @@ -186,10 +189,9 @@ class MasterSuite extends SparkFunSuite test("master correctly recover the application") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[FakeRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[FakeRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val fakeAppInfo = makeAppInfo(1024) val fakeWorkerInfo = makeWorkerInfo(8192, 16) @@ -286,8 +288,8 @@ class MasterSuite extends SparkFunSuite implicit val formats = org.json4s.DefaultFormats val reverseProxyUrl = "http://localhost:8080" val conf = new SparkConf() - conf.set("spark.ui.reverseProxy", "true") - conf.set("spark.ui.reverseProxyUrl", reverseProxyUrl) + conf.set(UI_REVERSE_PROXY, true) + conf.set(UI_REVERSE_PROXY_URL, reverseProxyUrl) val localCluster = new LocalSparkCluster(2, 2, 512, conf) localCluster.start() try { @@ -635,7 +637,7 @@ class MasterSuite extends SparkFunSuite } test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { - val conf = new SparkConf().set("spark.worker.timeout", "1") + val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) val master = makeMaster(conf) master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) eventually(timeout(10.seconds)) { diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 62fe0eaedfd27..2faf5ba54a832 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -24,6 +24,7 @@ import org.apache.commons.lang3.RandomUtils import org.apache.curator.test.TestingServer import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL import org.apache.spark.rpc.{RpcEndpoint, RpcEnv} import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.Utils @@ -51,7 +52,7 @@ class PersistenceEngineSuite extends SparkFunSuite { val zkTestServer = new TestingServer(findFreePort(conf)) try { testPersistenceEngine(conf, serializer => { - conf.set("spark.deploy.zookeeper.url", zkTestServer.getConnectString) + conf.set(ZOOKEEPER_URL, zkTestServer.getConnectString) new ZooKeeperPersistenceEngine(conf, serializer) }) } finally { diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 69a460fbc7dba..f4558aa3eb893 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -53,8 +53,11 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { } override def afterAll() { - masterWebUI.stop() - super.afterAll() + try { + masterWebUI.stop() + } finally { + super.afterAll() + } } test("kill application") { diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 75c50af23c66a..87655f310f1e9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -22,6 +22,7 @@ import java.lang.Boolean import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -93,7 +94,7 @@ class SubmitRestProtocolSuite extends SparkFunSuite { message.sparkProperties = conf.getAll.toMap message.validate() // optional fields - conf.set("spark.jars", "mayonnaise.jar,ketchup.jar") + conf.set(JARS, Seq("mayonnaise.jar", "ketchup.jar")) conf.set("spark.files", "fireball.png") conf.set("spark.driver.memory", s"${Utils.DEFAULT_DRIVER_MEM_MB}m") conf.set("spark.driver.cores", "180") diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index 2849a10a2c81e..e0e630e3be63b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -21,94 +21,36 @@ import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials -import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.util.Utils -class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { - private var delegationTokenManager: HadoopDelegationTokenManager = null - private var sparkConf: SparkConf = null - private var hadoopConf: Configuration = null +class HadoopDelegationTokenManagerSuite extends SparkFunSuite { + private val hadoopConf = new Configuration() - override def beforeAll(): Unit = { - super.beforeAll() - - sparkConf = new SparkConf() - hadoopConf = new Configuration() - } - - test("Correctly load default credential providers") { - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("bogus") should be (None) + test("default configuration") { + val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf) + assert(manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) + assert(manager.isProviderLoaded("hive")) } test("disable hive credential provider") { - sparkConf.set("spark.security.credentials.hive.enabled", "false") - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) + val sparkConf = new SparkConf(false).set("spark.security.credentials.hive.enabled", "false") + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) + assert(manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) + assert(!manager.isProviderLoaded("hive")) } test("using deprecated configurations") { - sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") - sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - } - - test("verify no credentials are obtained") { - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - val creds = new Credentials() - - // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. - delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) - val tokens = creds.getAllTokens - tokens.size() should be (0) - } - - test("obtain tokens For HiveMetastore") { - val hadoopConf = new Configuration() - hadoopConf.set("hive.metastore.kerberos.principal", "bob") - // thrift picks up on port 0 and bails out, without trying to talk to endpoint - hadoopConf.set("hive.metastore.uris", "http://localhost:0") - - val hiveCredentialProvider = new HiveDelegationTokenProvider() - val credentials = new Credentials() - hiveCredentialProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials) - - credentials.getAllTokens.size() should be (0) - } - - test("Obtain tokens For HBase") { - val hadoopConf = new Configuration() - hadoopConf.set("hbase.security.authentication", "kerberos") - - val hbaseTokenProvider = new HBaseDelegationTokenProvider() - val creds = new Credentials() - hbaseTokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, creds) - - creds.getAllTokens.size should be (0) + val sparkConf = new SparkConf(false) + .set("spark.yarn.security.tokens.hadoopfs.enabled", "false") + .set("spark.yarn.security.credentials.hive.enabled", "false") + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) + assert(!manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) + assert(!manager.isProviderLoaded("hive")) } test("SPARK-23209: obtain tokens when Hive classes are not available") { @@ -123,43 +65,41 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { throw new ClassNotFoundException(name) } - if (name.startsWith("java") || name.startsWith("scala")) { - currentLoader.loadClass(name) - } else { - val classFileName = name.replaceAll("\\.", "/") + ".class" - val in = currentLoader.getResourceAsStream(classFileName) - if (in != null) { - val bytes = IOUtils.toByteArray(in) - defineClass(name, bytes, 0, bytes.length) - } else { - throw new ClassNotFoundException(name) - } + val prefixBlacklist = Seq("java", "scala", "com.sun.", "sun.") + if (prefixBlacklist.exists(name.startsWith(_))) { + return currentLoader.loadClass(name) } + + val found = findLoadedClass(name) + if (found != null) { + return found + } + + val classFileName = name.replaceAll("\\.", "/") + ".class" + val in = currentLoader.getResourceAsStream(classFileName) + if (in != null) { + val bytes = IOUtils.toByteArray(in) + return defineClass(name, bytes, 0, bytes.length) + } + + throw new ClassNotFoundException(name) } } - try { - Thread.currentThread().setContextClassLoader(noHive) + Utils.withContextClassLoader(noHive) { val test = noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$")) test.getMethod("runTest").invoke(null) - } finally { - Thread.currentThread().setContextClassLoader(currentLoader) } } - - private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = { - Set(FileSystem.get(hadoopConf)) - } } /** Test code for SPARK-23209 to avoid using too much reflection above. */ -private object NoHiveTest extends Matchers { +private object NoHiveTest { def runTest(): Unit = { try { - val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), - _ => Set()) - manager.getServiceDelegationTokenProvider("hive") should be (None) + val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration()) + require(!manager.isProviderLoaded("hive")) } catch { case e: Throwable => // Throw a better exception in case the test fails, since there may be a lot of nesting. diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index e3fe2b696aa1f..f858f57d56ea7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.{Command, ExecutorState, ExternalShuffleService} import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged} import org.apache.spark.deploy.master.DriverState +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -100,7 +101,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (small number of executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 2.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -124,7 +125,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (more executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 30.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { @@ -157,7 +158,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (small number of drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 2.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -181,7 +182,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (more drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 30.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 77a7668d3a1d1..da8377e662e1a 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -39,6 +39,9 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.memory.MemoryManager +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.memory.TestMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.RDD import org.apache.spark.rpc.RpcEnv @@ -165,7 +168,7 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug val conf = new SparkConf() .setMaster("local") .setAppName("executor thread test") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext(conf) val executorThread = sc.parallelize(Seq(1), 1).map { _ => Thread.currentThread.getClass.getName diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index c88cc13654ce5..548949edf47bf 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.util.Utils class LauncherBackendSuite extends SparkFunSuite with Matchers { @@ -48,7 +49,7 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") .setMaster(master) .setAppResource(SparkLauncher.NO_RESOURCE) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index d56cfc183d921..5ce3453b682fe 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -248,7 +248,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val mm = UnifiedMemoryManager(conf, numCores = 1) // Try using an executor memory that's too small - val conf2 = conf.clone().set("spark.executor.memory", (reservedMemory / 2).toString) + val conf2 = conf.clone().set(EXECUTOR_MEMORY.key, (reservedMemory / 2).toString) val exception = intercept[IllegalArgumentException] { UnifiedMemoryManager(conf2, numCores = 1) } diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index a85011b42bbc7..800fc1e4a3f1d 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.METRICS_CONF class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { var filePath: String = _ @@ -31,7 +32,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with default properties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", "dummy-file") + sparkConf.set(METRICS_CONF, "dummy-file") val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -47,7 +48,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with properties set from a file") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -110,7 +111,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { setMetricsProperty(sparkConf, "*.source.jvm.class", "org.apache.spark.SomeOtherSource") setMetricsProperty(sparkConf, "master.sink.console.period", "50") setMetricsProperty(sparkConf, "master.sink.console.unit", "seconds") - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -135,7 +136,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with subProperties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index a7a24114f17e2..c512f29c8442b 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -35,7 +35,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile - conf = new SparkConf(false).set("spark.metrics.conf", filePath) + conf = new SparkConf(false).set(METRICS_CONF, filePath) securityMgr = new SecurityManager(conf) } diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 21138bd4a16ba..f1cf14de1f87d 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -32,6 +32,7 @@ import org.scalatest.Matchers import org.scalatest.mockito.MockitoSugar import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener @@ -50,8 +51,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on same password") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") testConnection(conf, conf) match { case Success(_) => // expected @@ -61,10 +62,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on mismatch password") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate.secret", "bad") + val conf1 = conf0.clone.set(AUTH_SECRET, "bad") testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Mismatched response") @@ -73,10 +74,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on server") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "false") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, false) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => // any funny error may occur, sever will interpret SASL token as RPC @@ -85,10 +86,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on client") { val conf0 = new SparkConf() - .set("spark.authenticate", "false") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, false) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "true") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, true) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Expected SaslMessage") @@ -97,8 +98,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security with aes encryption") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false") diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index a799b1cfb0765..86a74b52e9344 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -36,6 +36,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.config._ import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -693,42 +694,42 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { test("send with authentication") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("send with SASL encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("send with AES encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false")) } test("ask with authentication") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("ask with SASL encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("ask with AES encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false")) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 2155a0f2b6c21..5eef53f377194 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -180,7 +180,7 @@ class MapStatusSuite extends SparkFunSuite { test("SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE") { val conf = new SparkConf() - .set("spark.serializer", classOf[KryoSerializer].getName) + .set(config.SERIALIZER, classOf[KryoSerializer].getName) .setMaster("local") .setAppName("SPARK-21133") withSpark(new SparkContext(conf)) { sc => diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index aa9c36c0aaacb..8382d4e4fd46e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{Executor, TaskMetrics, TaskMetricsSuite} +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource import org.apache.spark.network.util.JavaUtils @@ -37,7 +38,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark test("provide metrics sources") { val filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile val conf = new SparkConf(loadDefaults = false) - .set("spark.metrics.conf", filePath) + .set(METRICS_CONF, filePath) sc = new SparkContext("local", "test", conf) val rdd = sc.makeRDD(1 to 1) val result = sc.runJob(rdd, (tc: TaskContext, it: Iterator[Int]) => { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index d264adaef90a5..f73ff67837c6d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -655,7 +655,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } test("abort the job if total size of results is too large") { - val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") + val conf = new SparkConf().set(config.MAX_RESULT_SIZE.key, "2m") sc = new SparkContext("local", "test", conf) def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => diff --git a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala index 3734f1cb408fe..8610b18702ec0 100644 --- a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala @@ -25,9 +25,10 @@ import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.generic.GenericData.Record import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.internal.config.SERIALIZER class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") val schema : Schema = SchemaBuilder .record("testRecord").fields() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index a1cf3570a7a6d..0d176f2595596 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -21,8 +21,11 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest._ -import org.apache.spark.util.Benchmark class KryoBenchmark extends SparkFunSuite { val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) @@ -129,9 +132,9 @@ class KryoBenchmark extends SparkFunSuite { def createSerializer(useUnsafe: Boolean): SerializerInstance = { val conf = new SparkConf() - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", useUnsafe.toString) + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, useUnsafe) new KryoSerializer(conf).newInstance() } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala new file mode 100644 index 0000000000000..c29a03afe5fee --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import scala.concurrent._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.util.ThreadUtils + + +/** + * Benchmark for KryoPool vs old "pool of 1". + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/KryoSerializerBenchmark-results.txt". + * }}} + */ +object KryoSerializerBenchmark extends BenchmarkBase { + + var sc: SparkContext = null + val N = 500 + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val name = "Benchmark KryoPool vs old\"pool of 1\" implementation" + runBenchmark(name) { + val benchmark = new Benchmark(name, N, 10, output = output) + Seq(true, false).foreach(usePool => run(usePool, benchmark)) + benchmark.run() + } + } + + private def run(usePool: Boolean, benchmark: Benchmark): Unit = { + lazy val sc = createSparkContext(usePool) + + benchmark.addCase(s"KryoPool:$usePool") { _ => + val futures = for (_ <- 0 until N) yield { + Future { + sc.parallelize(0 until 10).map(i => i + 1).count() + } + } + + val future = Future.sequence(futures) + + ThreadUtils.awaitResult(future, 10.minutes) + } + } + + def createSparkContext(usePool: Boolean): SparkContext = { + val conf = new SparkConf() + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_POOL, usePool) + + if (sc != null) { + sc.stop() + } + + sc = new SparkContext("local-cluster[4,1,1024]", "test", conf) + sc + } + + override def afterAll(): Unit = { + if (sc != null) { + sc.stop() + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 46aa9c37986cc..ae871091ba3c4 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -28,8 +28,8 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + .set(config.Kryo.KRYO_USER_REGISTRATORS, classOf[AppJarRegistrator].getName) .set(config.MAX_TASK_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala index cf01f79f49091..25f0b19c980fb 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -21,6 +21,8 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.LocalSparkContext._ import org.apache.spark.SparkContext import org.apache.spark.SparkException +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ class KryoSerializerResizableOutputSuite extends SparkFunSuite { @@ -29,9 +31,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo without resizable output buffer should fail on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "1m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "1m") withSpark(new SparkContext("local", "test", conf)) { sc => intercept[SparkException](sc.parallelize(x).collect()) } @@ -39,9 +41,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo with resizable output buffer should succeed on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "2m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "2m") withSpark(new SparkContext("local", "test", conf)) { sc => assert(sc.parallelize(x).collect() === x) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index fcb1315c13fb8..3f090f66cb6b3 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.serializer -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer +import java.util.concurrent.Executors import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException} @@ -29,19 +32,21 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", "false") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, false) test("SPARK-7392 configuration limits") { - val kryoBufferProperty = "spark.kryoserializer.buffer" - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferProperty = KRYO_SERIALIZER_BUFFER_SIZE.key + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key def newKryoInstance( conf: SparkConf, @@ -78,7 +83,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("basic types") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -111,7 +116,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("pairs") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -138,7 +143,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("Scala data structures") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -166,7 +171,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("Bug: SPARK-10251") { - val ser = new KryoSerializer(conf.clone.set("spark.kryo.registrationRequired", "true")) + val ser = new KryoSerializer(conf.clone.set(KRYO_REGISTRATION_REQUIRED, true)) .newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -250,7 +255,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { hashMap.put("foo", "bar") check(hashMap) - System.clearProperty("spark.kryo.registrator") + System.clearProperty(KRYO_USER_REGISTRATORS.key) } test("kryo with collect") { @@ -307,7 +312,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { import org.apache.spark.SparkException val conf = new SparkConf(false) - conf.set("spark.kryo.registrator", "this.class.does.not.exist") + conf.set(KRYO_USER_REGISTRATORS, "this.class.does.not.exist") val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) assert(thrown.getMessage.contains("Failed to register classes with Kryo")) @@ -334,7 +339,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("registration of HighlyCompressedMapStatus") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 // values, and they use a bitmap (dense) if they have more than 4096 values, and an @@ -352,7 +357,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("serialization buffer overflow reporting") { import org.apache.spark.SparkException - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key val largeObject = (1 to 1000000).toArray @@ -365,30 +370,6 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { assert(thrown.getCause.isInstanceOf[KryoException]) } - test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") { - val dir = Utils.createTempDir() - val tmpfile = dir.toString + "/RoaringBitmap" - val outStream = new FileOutputStream(tmpfile) - val output = new KryoOutput(outStream) - val bitmap = new RoaringBitmap - bitmap.add(1) - bitmap.add(3) - bitmap.add(5) - // Ignore Kryo because it doesn't use writeObject - bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) - output.flush() - output.close() - - val inStream = new FileInputStream(tmpfile) - val input = new KryoInput(inStream) - val ret = new RoaringBitmap - // Ignore Kryo because it doesn't use readObject - ret.deserialize(new KryoInputObjectInputBridge(null, input)) - input.close() - assert(ret == bitmap) - Utils.deleteRecursively(dir) - } - test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { val kryo = new KryoSerializer(conf).newKryo() @@ -406,7 +387,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("getAutoReset") { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] assert(!ser2.getAutoReset) @@ -432,11 +413,13 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { ser.deserialize[HashMap[Int, List[String]]](serializedMap) } - private def testSerializerInstanceReuse(autoReset: Boolean, referenceTracking: Boolean): Unit = { + private def testSerializerInstanceReuse( + autoReset: Boolean, referenceTracking: Boolean, usePool: Boolean): Unit = { val conf = new SparkConf(loadDefaults = false) - .set("spark.kryo.referenceTracking", referenceTracking.toString) + .set(KRYO_REFERENCE_TRACKING, referenceTracking) + .set(KRYO_USE_POOL, usePool) if (!autoReset) { - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) } val ser = new KryoSerializer(conf) val serInstance = ser.newInstance().asInstanceOf[KryoSerializerInstance] @@ -457,17 +440,74 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { // Regression test for SPARK-7766, an issue where disabling auto-reset and enabling // reference-tracking would lead to corrupted output when serializer instances are re-used - for (referenceTracking <- Set(true, false); autoReset <- Set(true, false)) { - test(s"instance reuse with autoReset = $autoReset, referenceTracking = $referenceTracking") { - testSerializerInstanceReuse(autoReset = autoReset, referenceTracking = referenceTracking) + for { + referenceTracking <- Seq(true, false) + autoReset <- Seq(true, false) + usePool <- Seq(true, false) + } { + test(s"instance reuse with autoReset = $autoReset, referenceTracking = $referenceTracking" + + s", usePool = $usePool") { + testSerializerInstanceReuse( + autoReset, referenceTracking, usePool) + } + } + + test("SPARK-25839 KryoPool implementation works correctly in multi-threaded environment") { + implicit val executionContext: ExecutionContext = ExecutionContext.fromExecutor( + Executors.newFixedThreadPool(4)) + + val ser = new KryoSerializer(conf.clone.set(KRYO_USE_POOL, true)) + + val tests = mutable.ListBuffer[Future[Boolean]]() + + def check[T: ClassTag](t: T) { + tests += Future { + val serializerInstance = ser.newInstance() + serializerInstance.deserialize[T](serializerInstance.serialize(t)) === t + } } + + check((1, 3)) + check(Array((1, 3))) + check(List((1, 3))) + check(List[Int]()) + check(List[Int](1, 2, 3)) + check(List[String]()) + check(List[String]("x", "y", "z")) + check(None) + check(Some(1)) + check(Some("hi")) + check(1 -> 1) + check(mutable.ArrayBuffer(1, 2, 3)) + check(mutable.ArrayBuffer("1", "2", "3")) + check(mutable.Map()) + check(mutable.Map(1 -> "one", 2 -> "two")) + check(mutable.Map("one" -> 1, "two" -> 2)) + check(mutable.HashMap(1 -> "one", 2 -> "two")) + check(mutable.HashMap("one" -> 1, "two" -> 2)) + check(List(Some(mutable.HashMap(1 -> 1, 2 -> 2)), None, Some(mutable.HashMap(3 -> 4)))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1 -> "one", 2 -> "two", 3 -> "three"))) + + tests.foreach { f => + assert(ThreadUtils.awaitResult(f, 10.seconds)) + } + } + + test("SPARK-27216: test RoaringBitmap ser/dser with Kryo") { + val expected = new RoaringBitmap() + expected.add(1787) + val ser = new KryoSerializer(conf).newInstance() + val actual: RoaringBitmap = ser.deserialize(ser.serialize(expected)) + assert(actual === expected) } } class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", classOf[KryoSerializer].getName) - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) - conf.set("spark.kryo.referenceTracking", "true") + conf.set(SERIALIZER, classOf[KryoSerializer].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_REFERENCE_TRACKING, true) conf.set("spark.shuffle.manager", "sort") conf.set("spark.shuffle.sort.bypassMergeThreshold", "200") diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index 99882bf76e29d..dad080c5fc161 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -24,6 +24,7 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset /** @@ -50,7 +51,7 @@ class SerializerPropertiesSuite extends SparkFunSuite { } test("KryoSerializer does not support relocation when auto-reset is disabled") { - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser = new KryoSerializer(conf) assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala index d63a45ae4a6a9..126ba0e8b1e93 100644 --- a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.serializer +import org.apache.spark.internal.config.Kryo._ + class UnsafeKryoSerializerSuite extends KryoSerializerSuite { // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. override def beforeAll() { - conf.set("spark.kryo.unsafe", "true") + conf.set(KRYO_USE_UNSAFE, true) super.beforeAll() } override def afterAll() { - conf.set("spark.kryo.unsafe", "false") + conf.set(KRYO_USE_UNSAFE, false) super.afterAll() } } diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 6214089126ae7..c63249ad4aae3 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 @@ -37,8 +38,6 @@ import org.apache.spark.util.kvstore._ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { - import config._ - private val conf = new SparkConf() .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) .set(ASYNC_TRACKING_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala index 07a7b58404c29..a99c1ec7e1f07 100644 --- a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala @@ -20,12 +20,11 @@ package org.apache.spark.status import org.mockito.Mockito._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Status._ import org.apache.spark.util.kvstore._ class ElementTrackingStoreSuite extends SparkFunSuite { - import config._ - test("tracking for multiple types") { val store = mock(classOf[KVStore]) val tracking = new ElementTrackingStore(store, new SparkConf() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 3962bdc27d22c..d0e90779fc009 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -31,7 +31,9 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config.{DRIVER_PORT, MEMORY_OFFHEAP_SIZE} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService @@ -85,8 +87,8 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite before { rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - conf.set("spark.authenticate", "false") - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(NETWORK_AUTH_ENABLED, false) + conf.set(DRIVER_PORT, rpcEnv.address.port) conf.set("spark.testing", "true") conf.set("spark.memory.fraction", "1") conf.set("spark.memory.storageFraction", "1") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index abde4dfd226c9..1f98ef2ca7aaf 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -122,7 +122,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set("spark.storage.unrollMemoryThreshold", "512") rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we // need to create a SparkContext is to initialize LiveListenerBus. diff --git a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala index 42828506895a7..fc16fe362882c 100644 --- a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage import org.apache.spark._ - +import org.apache.spark.internal.config._ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { /* Tests the ability of Spark to deal with user provided iterators from flatMap @@ -55,7 +55,7 @@ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { test("Serializer Reset") { val sconf = new SparkConf().setMaster("local").setAppName("serializer_reset_test") - .set("spark.serializer.objectStreamReset", "10") + .set(SERIALIZER_OBJECT_STREAM_RESET, 10) sc = new SparkContext(sconf) val expand_size = 500 val data = sc.parallelize(Seq(1, 2)). diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 6044563f7dde7..2945c3ee0a9d9 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -26,10 +26,10 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, StageData, StageStatus} -import org.apache.spark.status.config._ import org.apache.spark.ui.jobs.{ApiHelper, StagePage, StagesTab, TaskPagedTable} class StagePageSuite extends SparkFunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index e86cadfeebcff..b184b74bf3cb0 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -39,10 +39,11 @@ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.deploy.history.HistoryServerSuite -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Status._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus} -import org.apache.spark.status.config._ private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { @@ -103,9 +104,9 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.port", "0") - .set("spark.ui.killEnabled", killEnabled.toString) + .set(UI_ENABLED, true) + .set(UI_PORT, 0) + .set(UI_KILL_ENABLED, killEnabled) .set(MEMORY_OFFHEAP_SIZE.key, "64m") val sc = new SparkContext(conf) assert(sc.ui.isDefined) @@ -531,8 +532,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.port", "0") + .set(UI_ENABLED, true) + .set(UI_PORT, 0) .set(MAX_RETAINED_STAGES, 3) .set(MAX_RETAINED_JOBS, 2) .set(ASYNC_TRACKING_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 36ea3799afdf2..ef5e78cd8ae55 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -31,6 +31,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.util.Utils class UISuite extends SparkFunSuite { @@ -43,7 +44,7 @@ class UISuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 52cd5378bc715..242163931f7ac 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -34,7 +34,7 @@ import org.mockito.Mockito.{atLeast, mock, verify} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.logging.{FileAppender, RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy} class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { @@ -136,7 +136,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // setup input stream and appender val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) - val conf = new SparkConf().set(RollingFileAppender.RETAINED_FILES_PROPERTY, "10") + val conf = new SparkConf().set(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES, 10) val appender = new RollingFileAppender(testInputStream, testFile, new SizeBasedRollingPolicy(1000, false), conf, 10) @@ -200,13 +200,12 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { appender.awaitTermination() } - import RollingFileAppender._ - def rollingStrategy(strategy: String): Seq[(String, String)] = - Seq(STRATEGY_PROPERTY -> strategy) - def rollingSize(size: String): Seq[(String, String)] = Seq(SIZE_PROPERTY -> size) + Seq(config.EXECUTOR_LOGS_ROLLING_STRATEGY.key -> strategy) + def rollingSize(size: String): Seq[(String, String)] = + Seq(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> size) def rollingInterval(interval: String): Seq[(String, String)] = - Seq(INTERVAL_PROPERTY -> interval) + Seq(config.EXECUTOR_LOGS_ROLLING_TIME_INTERVAL.key -> interval) val msInDay = 24 * 60 * 60 * 1000L val msInHour = 60 * 60 * 1000L diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 39f4fba78583f..15076988bea1d 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TaskContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.SparkListener @@ -839,7 +840,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("isDynamicAllocationEnabled") { val conf = new SparkConf() conf.set("spark.master", "yarn") - conf.set("spark.submit.deployMode", "client") + conf.set(SUBMIT_DEPLOY_MODE, "client") assert(Utils.isDynamicAllocationEnabled(conf) === false) assert(Utils.isDynamicAllocationEnabled( conf.set("spark.dynamicAllocation.enabled", "false")) === false) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index cd25265784136..60e794af47b6e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -52,8 +52,8 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite val conf = new SparkConf(loadDefaults) // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + conf.set(SERIALIZER_OBJECT_STREAM_RESET, 1) + conf.set(SERIALIZER, "org.apache.spark.serializer.JavaSerializer") conf.set("spark.shuffle.spill.compress", codec.isDefined.toString) conf.set("spark.shuffle.compress", codec.isDefined.toString) codec.foreach { c => conf.set("spark.io.compression.codec", c) } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 47173b89e91e2..9425d82120d3b 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -23,6 +23,8 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.unsafe.array.LongArray @@ -267,12 +269,12 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def createSparkConf(loadDefaults: Boolean, kryo: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) if (kryo) { - conf.set("spark.serializer", classOf[KryoSerializer].getName) + conf.set(SERIALIZER, classOf[KryoSerializer].getName) } else { // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", classOf[JavaSerializer].getName) + conf.set(SERIALIZER_OBJECT_STREAM_RESET, 1) + conf.set(SERIALIZER, classOf[JavaSerializer].getName) } conf.set("spark.shuffle.sort.bypassMergeThreshold", "0") // Ensure that we actually have multiple batches per spill file diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index c0194c2964931..eb573c7d74f5e 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -1,5 +1,5 @@ JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar +RoaringBitmap-0.7.45.jar ST4-4.0.4.jar activation-1.1.1.jar aircompressor-0.10.jar @@ -176,6 +176,7 @@ scala-parser-combinators_2.11-1.1.0.jar scala-reflect-2.11.12.jar scala-xml_2.11-1.0.5.jar shapeless_2.11-2.3.2.jar +shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.15.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index de9638a2bf6bd..6f9276e59c32c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -1,5 +1,5 @@ JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar +RoaringBitmap-0.7.45.jar ST4-4.0.4.jar activation-1.1.1.jar aircompressor-0.10.jar @@ -140,7 +140,7 @@ libfb303-0.9.3.jar libthrift-0.9.3.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar -lz4-java-1.4.0.jar +lz4-java-1.5.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar @@ -177,6 +177,7 @@ scala-parser-combinators_2.11-1.1.0.jar scala-reflect-2.11.12.jar scala-xml_2.11-1.0.5.jar shapeless_2.11-2.3.2.jar +shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.15.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 83c378590550d..bdb86ab87c6a1 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -1,6 +1,6 @@ HikariCP-java7-2.4.12.jar JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar +RoaringBitmap-0.7.45.jar ST4-4.0.4.jar accessors-smart-1.2.jar activation-1.1.1.jar @@ -155,7 +155,7 @@ libfb303-0.9.3.jar libthrift-0.9.3.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar -lz4-java-1.4.0.jar +lz4-java-1.5.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar @@ -196,6 +196,7 @@ scala-parser-combinators_2.11-1.1.0.jar scala-reflect-2.11.12.jar scala-xml_2.11-1.0.5.jar shapeless_2.11-2.3.2.jar +shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.15.jar diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 02770439f5388..0db3d01a238d1 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -19,9 +19,9 @@ Please see [Spark Security](security.html) and the specific advice below before ## User Identity -Images built from the project provided Dockerfiles do not contain any [`USER`](https://docs.docker.com/engine/reference/builder/#user) directives. This means that the resulting images will be running the Spark processes as `root` inside the container. On unsecured clusters this may provide an attack vector for privilege escalation and container breakout. Therefore security conscious deployments should consider providing custom images with `USER` directives specifying an unprivileged UID and GID. +Images built from the project provided Dockerfiles contain a default [`USER`](https://docs.docker.com/engine/reference/builder/#user) directive with a default UID of `185`. This means that the resulting images will be running the Spark processes as this UID inside the container. Security conscious deployments should consider providing custom images with `USER` directives specifying their desired unprivileged UID and GID. The resulting UID should include the root group in its supplementary groups in order to be able to run the Spark executables. Users building their own images with the provided `docker-image-tool.sh` script can use the `-u ` option to specify the desired UID. -Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. +Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. This can be used to override the `USER` directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. ## Volume Mounts @@ -62,7 +62,8 @@ logs and remains in "completed" state in the Kubernetes API until it's eventuall Note that in the completed state, the driver pod does *not* use any computational or memory resources. -The driver and executor pod scheduling is handled by Kubernetes. It is possible to schedule the +The driver and executor pod scheduling is handled by Kubernetes. Communication to the Kubernetes API is done via fabric8, and we are +currently running kubernetes-client version 4.1.0. Make sure that when you are making infrastructure additions that you are aware of said version. It is possible to schedule the driver and executor pods on a subset of available nodes through a [node selector](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector) using the configuration property for it. It will be possible to use more advanced scheduling hints like [node/pod affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity) in a future release. @@ -86,6 +87,19 @@ Example usage is: $ ./bin/docker-image-tool.sh -r -t my-tag build $ ./bin/docker-image-tool.sh -r -t my-tag push ``` +This will build using the projects provided default `Dockerfiles`. To see more options available for customising the behaviour of this tool, including providing custom `Dockerfiles`, please run with the `-h` flag. + +By default `bin/docker-image-tool.sh` builds docker image for running JVM jobs. You need to opt-in to build additional +language binding docker images. + +Example usage is +```bash +# To build additional PySpark docker image +$ ./bin/docker-image-tool.sh -r -t my-tag -p ./kubernetes/dockerfiles/spark/bindings/python/Dockerfile build + +# To build additional SparkR docker image +$ ./bin/docker-image-tool.sh -r -t my-tag -R ./kubernetes/dockerfiles/spark/bindings/R/Dockerfile build +``` ## Cluster Mode @@ -202,6 +216,22 @@ To use a secret through an environment variable use the following options to the --conf spark.kubernetes.executor.secretKeyRef.ENV_NAME=name:key ``` +## Pod Template +Kubernetes allows defining pods from [template files](https://kubernetes.io/docs/concepts/workloads/pods/pod-overview/#pod-templates). +Spark users can similarly use template files to define the driver or executor pod configurations that Spark configurations do not support. +To do so, specify the spark properties `spark.kubernetes.driver.podTemplateFile` and `spark.kubernetes.executor.podTemplateFile` +to point to local files accessible to the `spark-submit` process. To allow the driver pod access the executor pod template +file, the file will be automatically mounted onto a volume in the driver pod when it's created. +Spark does not do any validation after unmarshalling these template files and relies on the Kubernetes API server for validation. + +It is important to note that Spark is opinionated about certain pod configurations so there are values in the +pod template that will always be overwritten by Spark. Therefore, users of this feature should note that specifying +the pod template file only lets Spark start with a template pod instead of an empty pod during the pod-building process. +For details, see the [full list](#pod-template-properties) of pod template values that will be overwritten by spark. + +Pod template files can also define multiple containers. In such cases, Spark will always assume that the first container in +the list will be the driver or executor container. + ## Using Kubernetes Volumes Starting with Spark 2.4.0, users can mount the following types of Kubernetes [volumes](https://kubernetes.io/docs/concepts/storage/volumes/) into the driver and executor pods: @@ -216,7 +246,8 @@ To mount a volume of any of the types above into the driver pod, use the followi ``` --conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path= --conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly= -``` +--conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath= +``` Specifically, `VolumeType` can be one of the following values: `hostPath`, `emptyDir`, and `persistentVolumeClaim`. `VolumeName` is the name you want to use for the volume under the `volumes` field in the pod specification. @@ -224,7 +255,7 @@ Each supported type of volumes may have some specific configuration options, whi ``` spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].options.[OptionName]= -``` +``` For example, the claim name of a `persistentVolumeClaim` with volume name `checkpointpvc` can be specified using the following property: @@ -232,7 +263,20 @@ For example, the claim name of a `persistentVolumeClaim` with volume name `check spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=check-point-pvc-claim ``` -The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. +The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. + +## Local Storage + +Spark uses temporary scratch space to spill data to disk during shuffles and other operations. When using Kubernetes as the resource manager the pods will be created with an [emptyDir](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir) volume mounted for each directory listed in `SPARK_LOCAL_DIRS`. If no directories are explicitly specified then a default directory is created and configured appropriately. + +`emptyDir` volumes use the ephemeral storage feature of Kubernetes and do not persist beyond the life of the pod. + +### Using RAM for local storage + +`emptyDir` volumes use the nodes backing storage for ephemeral storage by default, this behaviour may not be appropriate for some compute environments. For example if you have diskless nodes with remote storage mounted over a network, having lots of executors doing IO to this remote storage may actually degrade performance. + +In this case it may be desirable to set `spark.kubernetes.local.dirs.tmpfs=true` in your configuration which will cause the `emptyDir` volumes to be configured as `tmpfs` i.e. RAM backed volumes. When configured like this Sparks local storage usage will count towards your pods memory usage therefore you may wish to increase your memory requests by increasing the value of `spark.kubernetes.memoryOverheadFactor` as appropriate. + ## Introspection and Debugging @@ -287,6 +331,16 @@ the Spark application. ## Kubernetes Features +### Configuration File + +Your Kubernetes config file typically lives under `.kube/config` in your home directory or in a location specified by the `KUBECONFIG` environment variable. Spark on Kubernetes will attempt to use this file to do an initial auto-configuration of the Kubernetes client used to interact with the Kubernetes cluster. A variety of Spark configuration properties are provided that allow further customising the client configuration e.g. using an alternative authentication method. + +### Contexts + +Kubernetes configuration files can contain multiple contexts that allow for switching between different clusters and/or user identities. By default Spark on Kubernetes will use your current context (which can be checked by running `kubectl config current-context`) when doing the initial auto-configuration of the Kubernetes client. + +In order to use an alternative context users can specify the desired context via the Spark configuration property `spark.kubernetes.context` e.g. `spark.kubernetes.context=minikube`. + ### Namespaces Kubernetes has the concept of [namespaces](https://kubernetes.io/docs/concepts/overview/working-with-objects/namespaces/). @@ -346,6 +400,36 @@ RBAC authorization and how to configure Kubernetes service accounts for pods, pl [Using RBAC Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and [Configure Service Accounts for Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). +## Spark Application Management + +Kubernetes provides simple application management via the spark-submit CLI tool in cluster mode. +Users can kill a job by providing the submission ID that is printed when submitting their job. +The submission ID follows the format ``namespace:driver-pod-name``. +If user omits the namespace then the namespace set in current k8s context is used. +For example if user has set a specific namespace as follows `kubectl config set-context minikube --namespace=spark` +then the `spark` namespace will be used by default. On the other hand, if there is no namespace added to the specific context +then all namespaces will be considered by default. That means operations will affect all Spark applications matching the given submission ID regardless of namespace. +Moreover, spark-submit for application management uses the same backend code that is used for submitting the driver, so the same properties +like `spark.kubernetes.context` etc., can be re-used. + +For example: +```bash +$ spark-submit --kill spark:spark-pi-1547948636094-driver --master k8s://https://192.168.2.8:8443 +``` +Users also can list the application status by using the `--status` flag: + +```bash +$ spark-submit --status spark:spark-pi-1547948636094-driver --master k8s://https://192.168.2.8:8443 +``` +Both operations support glob patterns. For example user can run: +```bash +$ spark-submit --kill spark:spark-pi* --master k8s://https://192.168.2.8:8443 +``` +The above will kill all application with the specific prefix. + +User can specify the grace period for pod termination via the `spark.kubernetes.appKillPodDeletionGracePeriod` property, +using `--conf` as means to provide it (default value for all K8s pods is 30 secs). + ## Future Work There are several Spark on Kubernetes features that are currently being worked on or planned to be worked on. Those features are expected to eventually make it into future versions of the spark-kubernetes integration. @@ -354,18 +438,27 @@ Some of these include: * Dynamic Resource Allocation and External Shuffle Service * Local File Dependency Management -* Spark Application Management * Job Queues and Resource Management # Configuration -See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are -specific to Spark on Kubernetes. +See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are specific to Spark on Kubernetes. #### Spark Properties + + + + + @@ -695,7 +788,7 @@ specific to Spark on Kubernetes. @@ -764,6 +857,14 @@ specific to Spark on Kubernetes. spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. + + + + + @@ -788,6 +889,14 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. + + + + + @@ -804,19 +913,273 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=spark-pvc-claim. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.kubernetes.context(none) + The context from the user Kubernetes configuration file used for the initial + auto-configuration of the Kubernetes client library. When not specified then + the users current context is used. NB: Many of the + auto-configured settings can be overridden by the use of other Spark + configuration properties e.g. spark.kubernetes.namespace. +
spark.kubernetes.namespace default Specify the cpu request for each executor pod. Values conform to the Kubernetes convention. Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. - This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task + This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task parallelism, e.g., number of tasks an executor can run concurrently is not affected by this.
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath(none) + Specifies a subpath to be mounted from the volume into the driver pod. + spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. +
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly (none)
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.subPath(none) + Specifies a subpath to be mounted from the volume into the executor pod. + spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. +
spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.readOnly false
spark.kubernetes.local.dirs.tmpfsfalse + Configure the emptyDir volumes used to back SPARK_LOCAL_DIRS within the Spark driver and executor pods to use tmpfs backing i.e. RAM. See Local Storage earlier on this page + for more discussion of this. +
spark.kubernetes.memoryOverheadFactor 0.1 This sets the Memory Overhead Factor that will allocate memory to non-JVM memory, which includes off-heap memory allocations, non-JVM tasks, and various systems processes. For JVM-based jobs this value will default to 0.10 and 0.40 for non-JVM jobs. - This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with a higher default. + This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with a higher default.
spark.kubernetes.pyspark.pythonVersion "2" - This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. + This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. +
spark.kubernetes.kerberos.krb5.path(none) + Specify the local location of the krb5.conf file to be mounted on the driver and executors for Kerberos interaction. + It is important to note that the KDC defined needs to be visible from inside the containers. +
spark.kubernetes.kerberos.krb5.configMapName(none) + Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted on the driver and executors + for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also + be in the same namespace of the driver and executor pods. +
spark.kubernetes.hadoop.configMapName(none) + Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, to be mounted on the driver + and executors for custom Hadoop configuration. +
spark.kubernetes.kerberos.tokenSecret.name(none) + Specify the name of the secret where your existing delegation tokens are stored. This removes the need for the job user + to provide any kerberos credentials for launching a job. +
spark.kubernetes.kerberos.tokenSecret.itemKey(none) + Specify the item key of the data where your existing delegation tokens are stored. This removes the need for the job user + to provide any kerberos credentials for launching a job. +
spark.kubernetes.driver.podTemplateFile(none) + Specify the local file that contains the driver [pod template](#pod-template). For example + spark.kubernetes.driver.podTemplateFile=/path/to/driver-pod-template.yaml` +
spark.kubernetes.executor.podTemplateFile(none) + Specify the local file that contains the executor [pod template](#pod-template). For example + spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml` +
spark.kubernetes.executor.deleteOnTerminationtrue + Specify whether executor pods should be deleted in case of failure or normal termination. +
spark.kubernetes.submission.connectionTimeout10000 + Connection timeout in milliseconds for the kubernetes client to use for starting the driver. +
spark.kubernetes.submission.requestTimeout10000 + Request timeout in milliseconds for the kubernetes client to use for starting the driver. +
spark.kubernetes.driver.connectionTimeout10000 + Connection timeout in milliseconds for the kubernetes client in driver to use when requesting executors. +
spark.kubernetes.driver.requestTimeout10000 + Request timeout in milliseconds for the kubernetes client in driver to use when requesting executors. +
spark.kubernetes.appKillPodDeletionGracePeriod(none) + Specify the grace period in seconds when deleting a Spark application using spark-submit. +
+ +#### Pod template properties + +See the below table for the full list of pod specifications that will be overwritten by spark. + +### Pod Metadata + + + + + + + + + + + + + + + + + + + + + + + +
Pod metadata keyModified valueDescription
nameValue of spark.kubernetes.driver.pod.name + The driver pod name will be overwritten with either the configured or default value of + spark.kubernetes.driver.pod.name. The executor pod names will be unaffected. +
namespaceValue of spark.kubernetes.namespace + Spark makes strong assumptions about the driver and executor namespaces. Both driver and executor namespaces will + be replaced by either the configured or default spark conf value. +
labelsAdds the labels from spark.kubernetes.{driver,executor}.label.* + Spark will add additional labels specified by the spark configuration. +
annotationsAdds the annotations from spark.kubernetes.{driver,executor}.annotation.* + Spark will add additional labels specified by the spark configuration. +
+ +### Pod Spec + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Pod spec keyModified valueDescription
imagePullSecretsAdds image pull secrets from spark.kubernetes.container.image.pullSecrets + Additional pull secrets will be added from the spark configuration to both executor pods. +
nodeSelectorAdds node selectors from spark.kubernetes.node.selector.* + Additional node selectors will be added from the spark configuration to both executor pods. +
restartPolicy"never" + Spark assumes that both drivers and executors never restart. +
serviceAccountValue of spark.kubernetes.authenticate.driver.serviceAccountName + Spark will override serviceAccount with the value of the spark configuration for only + driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected. +
serviceAccountNameValue of spark.kubernetes.authenticate.driver.serviceAccountName + Spark will override serviceAccountName with the value of the spark configuration for only + driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected. +
volumesAdds volumes from spark.kubernetes.{driver,executor}.volumes.[VolumeType].[VolumeName].mount.path + Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing + spark conf and pod template files. +
+ +### Container spec + +The following affect the driver and executor containers. All other containers in the pod spec will be unaffected. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Container spec keyModified valueDescription
envAdds env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName] + Spark will add driver env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName], and + executor env variables from spark.executorEnv.[EnvironmentVariableName]. +
imageValue of spark.kubernetes.{driver,executor}.container.image + The image will be defined by the spark configurations. +
imagePullPolicyValue of spark.kubernetes.container.image.pullPolicy + Spark will override the pull policy for both driver and executors. +
nameSee description. + The container name will be assigned by spark ("spark-kubernetes-driver" for the driver container, and + "executor" for each executor container) if not defined by the pod template. If the container is defined by the + template, the template's name will be used. +
resourcesSee description + The cpu limits are set by spark.kubernetes.{driver,executor}.limit.cores. The cpu is set by + spark.{driver,executor}.cores. The memory request and limit are set by summing the values of + spark.{driver,executor}.memory and spark.{driver,executor}.memoryOverhead. +
volumeMountsAdd volumes from spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.{path,readOnly} + Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing + spark conf and pod template files.
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f265075e351e0..854aa90dd9d8e 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -126,6 +126,43 @@ To use a custom metrics.properties for the application master and executors, upd Use lower-case suffixes, e.g. k, m, g, t, and p, for kibi-, mebi-, gibi-, tebi-, and pebibytes, respectively. + + spark.yarn.am.resource.{resource-type} + (none) + + Amount of resource to use for the YARN Application Master in client mode. + In cluster mode, use spark.yarn.driver.resource.<resource-type> instead. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

+ Example: + To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu + + + + spark.yarn.driver.resource.{resource-type} + (none) + + Amount of resource to use for the YARN Application Master in cluster mode. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

+ Example: + To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu + + + + spark.yarn.executor.resource.{resource-type} + (none) + + Amount of resource to use per executor process. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

+ Example: + To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu + + spark.yarn.am.cores 1 diff --git a/docs/security.md b/docs/security.md index 2948fbcb7d9d4..6c6c6aef2d392 100644 --- a/docs/security.md +++ b/docs/security.md @@ -26,21 +26,29 @@ not documented, Spark does not support. Spark currently supports authentication for RPC channels using a shared secret. Authentication can be turned on by setting the `spark.authenticate` configuration parameter. -The exact mechanism used to generate and distribute the shared secret is deployment-specific. +The exact mechanism used to generate and distribute the shared secret is deployment-specific. Unless +specified below, the secret must be defined by setting the `spark.authenticate.secret` config +option. The same secret is shared by all Spark applications and daemons in that case, which limits +the security of these deployments, especially on multi-tenant clusters. -For Spark on [YARN](running-on-yarn.html) and local deployments, Spark will automatically handle -generating and distributing the shared secret. Each application will use a unique shared secret. In +The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should +ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 +respectively by default) are restricted to hosts that are trusted to submit jobs. + +### YARN + +For Spark on [YARN](running-on-yarn.html), Spark will automatically handle generating and +distributing the shared secret. Each application will use a unique shared secret. In the case of YARN, this feature relies on YARN RPC encryption being enabled for the distribution of secrets to be secure. -For other resource managers, `spark.authenticate.secret` must be configured on each of the nodes. -This secret will be shared by all the daemons and applications, so this deployment configuration is -not as secure as the above, especially when considering multi-tenant clusters. In this -configuration, a user with the secret can effectively impersonate any other user. +### Kubernetes -The Rest Submission Server and the MesosClusterDispatcher do not support authentication. You should -ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 -respectively by default) are restricted to hosts that are trusted to submit jobs. +On Kubernetes, Spark will also automatically generate an authentication secret unique to each +application. The secret is propagated to executor pods using environment variables. This means +that any user that can list pods in the namespace where the Spark application is running can +also see their authentication secret. Access control rules should be properly set up by the +Kubernetes admin to ensure that Spark authentication is secure. @@ -58,6 +66,50 @@ respectively by default) are restricted to hosts that are trusted to submit jobs
Property NameDefaultMeaning
+Alternatively, one can mount authentication secrets using files and Kubernetes secrets that +the user mounts into their pods. + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.authenticate.secret.fileNone + Path pointing to the secret key to use for securing connections. Ensure that the + contents of the file have been securely generated. This file is loaded on both the driver + and the executors unless other settings override this (see below). +
spark.authenticate.secret.driver.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark driver reads to load the secret. + Useful when in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.executor.file must be specified so that the driver + and the executors can both use files to load the secret key. Ensure that the contents of the file + on the driver is identical to the contents of the file on the executors. +
spark.authenticate.secret.executor.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark executors read to load the secret. + Useful in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.driver.file must be specified so that the driver + and the executors can both use files to load the secret key. Ensure that the contents of the file + on the driver is identical to the contents of the file on the executors. +
+ +Note that when using files, Spark will not mount these files into the containers for you. It is up +you to ensure that the secret files are deployed securely into your containers and that the driver's +secret file agrees with the executors' secret file. + ## Encryption Spark supports AES-based encryption for RPC connections. For encryption to be enabled, RPC @@ -735,7 +787,82 @@ with encryption, at least. The Kerberos login will be periodically renewed using the provided credentials, and new delegation tokens for supported will be created. +## Secure Interaction with Kubernetes + +When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens +so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are +shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: + +In all cases you must define the environment variable: `HADOOP_CONF_DIR` or +`spark.kubernetes.hadoop.configMapName.` + +It also important to note that the KDC needs to be visible from inside the containers. + +If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be +achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing ConfigMap. + +1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: +```bash +/usr/bin/kinit -kt / +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` +2. Submitting with a local Keytab and Principal +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kerberos.keytab= \ + --conf spark.kerberos.principal= \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` +3. Submitting with pre-populated secrets, that contain the Delegation Token, already existing within the namespace +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` + +3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap and pre-created `HADOOP_CONF_DIR` ConfigMap +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.hadoop.configMapName= \ + --conf spark.kubernetes.kerberos.krb5.configMapName= \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` # Event Logging If your applications are using event logging, the directory where the event logs go diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 1a779716ec4c0..a73859891a860 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -22,6 +22,9 @@ import java.io.File import scala.io.Source._ +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.SparkSession /** @@ -107,6 +110,13 @@ object DFSReadWriteTest { println("Writing local file to DFS") val dfsFilename = s"$dfsDirPath/dfs_read_write_test" + + // delete file if exists + val fs = FileSystem.get(spark.sessionState.newHadoopConf()) + if (fs.exists(new Path(dfsFilename))) { + fs.delete(new Path(dfsFilename), true) + } + val fileRDD = spark.sparkContext.parallelize(fileContents) fileRDD.saveAsTextFile(dfsFilename) @@ -123,7 +133,6 @@ object DFSReadWriteTest { .sum spark.stop() - if (localWordCount == dfsWordCount) { println(s"Success! Local Word Count $localWordCount and " + s"DFS Word Count $dfsWordCount agree.") @@ -131,7 +140,6 @@ object DFSReadWriteTest { println(s"Failure! Local Word Count $localWordCount " + s"and DFS Word Count $dfsWordCount disagree.") } - } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index e1f985ece8c06..08af3306a96f1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -41,6 +41,8 @@ object HdfsTest { val end = System.currentTimeMillis() println(s"Iteration $iter took ${end-start} ms") } + println(s"File contents: ${file.map(_.toString).take(1).mkString(",").slice(0, 10)}") + println(s"Returned length(s) of: ${file.map(_.length).sum().toString}") spark.stop() } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 4324cc6d0f804..9241b13c100f1 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -50,13 +50,18 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with val utils = new PollingFlumeTestUtils override def beforeAll(): Unit = { + super.beforeAll() _sc = new SparkContext(conf) } override def afterAll(): Unit = { - if (_sc != null) { - _sc.stop() - _sc = null + try { + if (_sc != null) { + _sc.stop() + _sc = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index eb186970fc25d..8cfca56433f5d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -48,9 +48,12 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest } override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null + try { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + } finally { super.afterAll() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index a2213e024bd98..81832fbdcd7ec 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -48,9 +48,12 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { } override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null + try { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + } finally { super.afterAll() } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 661b67a8ab68a..1974bb1e12e15 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -59,14 +59,19 @@ class DirectKafkaStreamSuite private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 3ac6509b04707..561bca5f55370 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -44,20 +44,27 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private var sc: SparkContext = _ override def beforeAll { + super.beforeAll() sc = new SparkContext(sparkConf) kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (sc != null) { - sc.stop - sc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (sc != null) { + sc.stop + sc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index ecca38784e777..3fd37f4c8ac90 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -57,14 +57,19 @@ class DirectKafkaStreamSuite private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index d66830cbacdee..73d528518d486 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -32,6 +32,7 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll() { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -41,9 +42,13 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { } override def afterAll() { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 809699a739962..72f954149fefe 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -35,20 +35,27 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private var sc: SparkContext = _ override def beforeAll { + super.beforeAll() sc = new SparkContext(sparkConf) kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (sc != null) { - sc.stop - sc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (sc != null) { + sc.stop + sc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 426cd83b4ddf8..ed130f5990955 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -35,19 +35,26 @@ class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll(): Unit = { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - ssc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (ssc != null) { + ssc.stop() + ssc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 57f89cc7dbc65..5da5ea49d77ed 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -51,6 +51,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite private var tempDirectory: File = null override def beforeAll(): Unit = { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -65,11 +66,15 @@ class ReliableKafkaStreamSuite extends SparkFunSuite } override def afterAll(): Unit = { - Utils.deleteRecursively(tempDirectory) + try { + Utils.deleteRecursively(tempDirectory) - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala index e0e26847aa0ec..361520e292266 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala @@ -40,7 +40,11 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE .checkpointAppName(checkpointAppName) override def afterAll(): Unit = { - ssc.stop() + try { + ssc.stop() + } finally { + super.afterAll() + } } test("should raise an exception if the StreamingContext is missing") { diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index a7a68eba910bf..6d27445c5b606 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -71,17 +71,21 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun } override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - } - if (sc != null) { - sc.stop() - } - if (testUtils != null) { - // Delete the Kinesis stream as well as the DynamoDB table generated by - // Kinesis Client Library when consuming the stream - testUtils.deleteStream() - testUtils.deleteDynamoDBTable(appName) + try { + if (ssc != null) { + ssc.stop() + } + if (sc != null) { + sc.stop() + } + if (testUtils != null) { + // Delete the Kinesis stream as well as the DynamoDB table generated by + // Kinesis Client Library when consuming the stream + testUtils.deleteStream() + testUtils.deleteDynamoDBTable(appName) + } + } finally { + super.afterAll() } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index 6355e0f179496..a26d3069330aa 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.ml.attribute import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.types._ class AttributeSuite extends SparkFunSuite { @@ -221,4 +224,20 @@ class AttributeSuite extends SparkFunSuite { val decimalFldWithMeta = new StructField("x", DecimalType(38, 18), false, metadata) assert(Attribute.fromStructField(decimalFldWithMeta).isNumeric) } + + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set(KRYO_REGISTRATION_REQUIRED, true) + + val ser = new KryoSerializer(conf).newInstance() + + val numericAttr = new NumericAttribute(Some("numeric"), Some(1), Some(1.0), Some(2.0)) + val nominalAttr = new NominalAttribute(Some("nominal"), Some(2), Some(false)) + val binaryAttr = new BinaryAttribute(Some("binary"), Some(3), Some(Array("i", "j"))) + + Seq(numericAttr, nominalAttr, binaryAttr).foreach { i => + val i2 = ser.deserialize[Attribute](ser.serialize(i)) + assert(i === i2) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index cca7399b4b9c5..5a74490058398 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class InstanceSuite extends SparkFunSuite{ test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala index 05c7a58ee5ffd..63c163590af59 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala index f41abe48f2c58..3a44e79291cc3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer class TreePointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 1b98250061c7a..3ba5d0ce45d7c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ @@ -316,7 +317,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index f4fa216b8eba0..713dd276edc9d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.util.Utils class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -109,12 +111,13 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { test("big model load / save") { // backupping old values - val oldBufferConfValue = spark.conf.get("spark.kryoserializer.buffer.max", "64m") - val oldBufferMaxConfValue = spark.conf.get("spark.kryoserializer.buffer", "64k") + val oldBufferConfValue = spark.conf.get(KRYO_SERIALIZER_BUFFER_SIZE.key, "64m") + val oldBufferMaxConfValue = spark.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "64k") // setting test values to trigger partitioning - spark.conf.set("spark.kryoserializer.buffer", "50b") - spark.conf.set("spark.kryoserializer.buffer.max", "50b") + + // this is needed to set configurations which are also defined to SparkConf + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "50b") // create a model bigger than 50 Bytes val word2VecMap = Map((0 to 10).map(i => s"$i" -> Array.fill(10)(0.1f)): _*) @@ -137,8 +140,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { "that spans over multiple partitions", t) } finally { Utils.deleteRecursively(tempDir) - spark.conf.set("spark.kryoserializer.buffer", oldBufferConfValue) - spark.conf.set("spark.kryoserializer.buffer.max", oldBufferMaxConfValue) + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, oldBufferConfValue) + spark.conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, oldBufferMaxConfValue) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index d76edb940b2bd..22eb49cfb5126 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -34,7 +35,7 @@ import org.apache.spark.serializer.KryoSerializer class MatricesSuite extends SparkFunSuite { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala index 5973479dfb5ed..e2976e1ab022b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.linalg +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.util.Benchmark /** * Serialization benchmark for VectorUDT. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 217b4a35438fd..fee0b02bf8ed8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -25,6 +25,7 @@ import org.json4s.jackson.JsonMethods.{parse => parseJson} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -38,7 +39,7 @@ class VectorsSuite extends SparkFunSuite with Logging { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index c1449ece740d4..d3366dcde2ede 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.serializer.KryoSerializer @@ -57,7 +58,7 @@ class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala index 669d44223d713..bf277f51af68d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.mllib.stat.distribution import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ diff --git a/pom.xml b/pom.xml index 5fe84cd11a3a5..f31278d9d3e37 100644 --- a/pom.xml +++ b/pom.xml @@ -574,7 +574,7 @@ org.roaringbitmap RoaringBitmap - 0.5.11 + 0.7.45 commons-net @@ -2011,6 +2011,11 @@ + + io.growing + bitmap + 0.6.26-GIO1.8.9 + @@ -2619,6 +2624,13 @@ + + + releases + http://54.223.132.19:8081/nexus/content/repositories/releases + + + + + ${project.build.directory}/spark-dist-unpacked N/A ${project.build.directory}/imageTag.txt minikube docker.io/kubespark + + + + + @@ -135,9 +144,13 @@ ${spark.kubernetes.test.unpackSparkDir} ${spark.kubernetes.test.imageRepo} ${spark.kubernetes.test.deployMode} + ${spark.kubernetes.test.kubeConfigContext} ${spark.kubernetes.test.master} ${spark.kubernetes.test.namespace} ${spark.kubernetes.test.serviceAccountName} + ${spark.kubernetes.test.jvmImage} + ${spark.kubernetes.test.pythonImage} + ${spark.kubernetes.test.rImage} ${test.exclude.tags} ${test.include.tags} @@ -145,14 +158,10 @@ test + none test - - - (?<!Suite) - integration-test diff --git a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh index ccfb8e767c529..84c42cb19ef5c 100755 --- a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh +++ b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh @@ -16,6 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +set -ex TEST_ROOT_DIR=$(git rev-parse --show-toplevel) UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked" IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt" @@ -58,32 +59,64 @@ while (( "$#" )); do shift done -if [[ $SPARK_TGZ == "N/A" ]]; +rm -rf "$UNPACKED_SPARK_TGZ" +if [[ $SPARK_TGZ == "N/A" && $IMAGE_TAG == "N/A" ]]; then - echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1; + # If there is no spark image tag to test with and no src dir, build from current + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + SPARK_INPUT_DIR="$(cd "$SCRIPT_DIR/"../../../../ >/dev/null 2>&1 && pwd )" + DOCKER_FILE_BASE_PATH="$SPARK_INPUT_DIR/resource-managers/kubernetes/docker/src/main/dockerfiles/spark" +elif [[ $IMAGE_TAG == "N/A" ]]; +then + # If there is a test src tarball and no image tag we will want to build from that + mkdir -p $UNPACKED_SPARK_TGZ + tar -xzvf $SPARK_TGZ --strip-components=1 -C $UNPACKED_SPARK_TGZ; + SPARK_INPUT_DIR="$UNPACKED_SPARK_TGZ" + DOCKER_FILE_BASE_PATH="$SPARK_INPUT_DIR/kubernetes/dockerfiles/spark" fi -rm -rf $UNPACKED_SPARK_TGZ -mkdir -p $UNPACKED_SPARK_TGZ -tar -xzvf $SPARK_TGZ --strip-components=1 -C $UNPACKED_SPARK_TGZ; +# If there is a specific Spark image skip building and extraction/copy if [[ $IMAGE_TAG == "N/A" ]]; then IMAGE_TAG=$(uuidgen); - cd $UNPACKED_SPARK_TGZ - if [[ $DEPLOY_MODE == cloud ]] ; - then - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG build - if [[ $IMAGE_REPO == gcr.io* ]] ; - then - gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG - else - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push - fi - else - # -m option for minikube. - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG build - fi + cd $SPARK_INPUT_DIR + + # Build PySpark image + LANGUAGE_BINDING_BUILD_ARGS="-p $DOCKER_FILE_BASE_PATH/bindings/python/Dockerfile" + + # Build SparkR image -- disabled since this fails, re-enable as part of SPARK-25152 + # LANGUAGE_BINDING_BUILD_ARGS="$LANGUAGE_BINDING_BUILD_ARGS -R $DOCKER_FILE_BASE_PATH/bindings/R/Dockerfile" + + case $DEPLOY_MODE in + cloud) + # Build images + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + + # Push images appropriately + if [[ $IMAGE_REPO == gcr.io* ]] ; + then + gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG + else + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push + fi + ;; + + docker-for-desktop) + # Only need to build as this will place it in our local Docker repo which is all + # we need for Docker for Desktop to work so no need to also push + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + ;; + + minikube) + # Only need to build and if we do this with the -m option for minikube we will + # build the images directly using the minikube Docker daemon so no need to push + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + ;; + *) + echo "Unrecognized deploy mode $DEPLOY_MODE" && exit 1 + ;; + esac cd - fi diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml new file mode 100644 index 0000000000000..0c185be81d59e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml @@ -0,0 +1,26 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +apiVersion: v1 +Kind: Pod +metadata: + labels: + template-label-key: driver-template-label-value +spec: + containers: + - name: test-driver-container + image: will-be-overwritten + diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml new file mode 100644 index 0000000000000..0282e23a39bd2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml @@ -0,0 +1,25 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +apiVersion: v1 +Kind: Pod +metadata: + labels: + template-label-key: executor-template-label-value +spec: + containers: + - name: test-executor-container + image: will-be-overwritten diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala index 4e749c40563dc..3c1d9ea54ab2c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala @@ -52,6 +52,9 @@ private[spark] trait BasicTestsSuite { k8sSuite: KubernetesSuite => } test("Run SparkPi with an argument.", k8sTestTag) { + // This additional configuration with snappy is for SPARK-26995 + sparkAppConf + .set("spark.io.compression.codec", "snappy") runSparkPiAndVerifyCompletion(appArgs = Array("5")) } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala index c8bd584516ea5..2720cdf74ca8f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala @@ -62,11 +62,12 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => .endMetadata() .withNewSpec() .withServiceAccountName(kubernetesTestComponents.serviceAccountName) + .withRestartPolicy("Never") .addNewContainer() .withName("spark-example") .withImage(image) .withImagePullPolicy("IfNotPresent") - .withCommand("/opt/spark/bin/run-example") + .addToArgs("/opt/spark/bin/run-example") .addToArgs("--master", s"k8s://https://kubernetes.default.svc") .addToArgs("--deploy-mode", "client") .addToArgs("--conf", s"spark.kubernetes.container.image=$image") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index e6840ce818c1f..bc0bb20908254 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -19,9 +19,11 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.File import java.nio.file.{Path, Paths} import java.util.UUID -import java.util.regex.Pattern -import com.google.common.io.PatternFilenameFilter +import scala.collection.JavaConverters._ + +import com.google.common.base.Charsets +import com.google.common.io.Files import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action @@ -29,23 +31,23 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, Tag} import org.scalatest.Matchers import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} -import scala.collection.JavaConverters._ -import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.k8s.integrationtest.TestConfig._ +import org.apache.spark.{SPARK_VERSION, SparkFunSuite} +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ -private[spark] class KubernetesSuite extends SparkFunSuite +class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite - with PythonTestsSuite with ClientModeTestsSuite + with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite with Logging with Eventually with Matchers { import KubernetesSuite._ - private var sparkHomeDir: Path = _ - private var pyImage: String = _ - private var rImage: String = _ + protected var sparkHomeDir: Path = _ + protected var pyImage: String = _ + protected var rImage: String = _ protected var image: String = _ protected var testBackend: IntegrationTestBackend = _ @@ -66,7 +68,32 @@ private[spark] class KubernetesSuite extends SparkFunSuite private val extraExecTotalMemory = s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}Mi" + /** + * Build the image ref for the given image name, taking the repo and tag from the + * test configuration. + */ + private def testImageRef(name: String): String = { + val tag = sys.props.get(CONFIG_KEY_IMAGE_TAG_FILE) + .map { path => + val tagFile = new File(path) + require(tagFile.isFile, + s"No file found for image tag at ${tagFile.getAbsolutePath}.") + Files.toString(tagFile, Charsets.UTF_8).trim + } + .orElse(sys.props.get(CONFIG_KEY_IMAGE_TAG)) + .getOrElse { + throw new IllegalArgumentException( + s"One of $CONFIG_KEY_IMAGE_TAG_FILE or $CONFIG_KEY_IMAGE_TAG is required.") + } + val repo = sys.props.get(CONFIG_KEY_IMAGE_REPO) + .map { _ + "/" } + .getOrElse("") + + s"$repo$name:$tag" + } + override def beforeAll(): Unit = { + super.beforeAll() // The scalatest-maven-plugin gives system properties that are referenced but not set null // values. We need to remove the null-value properties before initializing the test backend. val nullValueProperties = System.getProperties.asScala @@ -76,29 +103,40 @@ private[spark] class KubernetesSuite extends SparkFunSuite System.clearProperty(key) } - val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir") - require(sparkDirProp != null, "Spark home directory must be provided in system properties.") + val possible_spark_dirs = List( + // If someone specified the tgz for the tests look at the extraction dir + System.getProperty(CONFIG_KEY_UNPACK_DIR), + // Try the spark test home + sys.props("spark.test.home") + ) + val sparkDirProp = possible_spark_dirs.filter(x => + new File(Paths.get(x).toFile, "bin/spark-submit").exists).headOption.getOrElse(null) + require(sparkDirProp != null, + s"Spark home directory must be provided in system properties tested $possible_spark_dirs") sparkHomeDir = Paths.get(sparkDirProp) require(sparkHomeDir.toFile.isDirectory, s"No directory found for spark home specified at $sparkHomeDir.") - val imageTag = getTestImageTag - val imageRepo = getTestImageRepo - image = s"$imageRepo/spark:$imageTag" - pyImage = s"$imageRepo/spark-py:$imageTag" - rImage = s"$imageRepo/spark-r:$imageTag" - - val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars")) - .toFile - .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0) - containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" + - s"${sparkDistroExamplesJarFile.getName}" + image = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_JVM, "spark")) + pyImage = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_PYTHON, "spark-py")) + rImage = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_R, "spark-r")) + + val scalaVersion = scala.util.Properties.versionNumberString + .split("\\.") + .take(2) + .mkString(".") + containerLocalSparkDistroExamplesJar = + s"local:///opt/spark/examples/jars/spark-examples_$scalaVersion-${SPARK_VERSION}.jar" testBackend = IntegrationTestBackendFactory.getTestBackend testBackend.initialize() kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient) } override def afterAll(): Unit = { - testBackend.cleanUp() + try { + testBackend.cleanUp() + } finally { + super.afterAll() + } } before { @@ -109,6 +147,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite .set("spark.kubernetes.driver.pod.name", driverPodName) .set("spark.kubernetes.driver.label.spark-app-locator", appLocator) .set("spark.kubernetes.executor.label.spark-app-locator", appLocator) + .set(NETWORK_AUTH_ENABLED.key, "true") if (!kubernetesTestComponents.hasUserSpecifiedNamespace) { kubernetesTestComponents.createNamespace() } @@ -139,6 +178,29 @@ private[spark] class KubernetesSuite extends SparkFunSuite isJVM) } + protected def runDFSReadWriteAndVerifyCompletion( + wordCount: Int, + appResource: String = containerLocalSparkDistroExamplesJar, + driverPodChecker: Pod => Unit = doBasicDriverPodCheck, + executorPodChecker: Pod => Unit = doBasicExecutorPodCheck, + appArgs: Array[String] = Array.empty[String], + appLocator: String = appLocator, + isJVM: Boolean = true, + interval: Option[PatienceConfiguration.Interval] = None): Unit = { + runSparkApplicationAndVerifyCompletion( + appResource, + SPARK_DFS_READ_WRITE_TEST, + Seq(s"Success! Local Word Count $wordCount and " + + s"DFS Word Count $wordCount agree."), + appArgs, + driverPodChecker, + executorPodChecker, + appLocator, + isJVM, + None, + interval) + } + protected def runSparkRemoteCheckAndVerifyCompletion( appResource: String = containerLocalSparkDistroExamplesJar, driverPodChecker: Pod => Unit = doBasicDriverPodCheck, @@ -202,7 +264,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite executorPodChecker: Pod => Unit, appLocator: String, isJVM: Boolean, - pyFiles: Option[String] = None): Unit = { + pyFiles: Option[String] = None, + interval: Option[PatienceConfiguration.Interval] = None): Unit = { val appArguments = SparkAppArguments( mainAppResource = appResource, mainClass = mainClass, @@ -242,10 +305,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite } } }) - Eventually.eventually(TIMEOUT, INTERVAL) { execPods.values.nonEmpty should be (true) } + + val patienceInterval = interval.getOrElse(INTERVAL) + Eventually.eventually(TIMEOUT, patienceInterval) { execPods.values.nonEmpty should be (true) } execWatcher.close() execPods.values.foreach(executorPodChecker(_)) - Eventually.eventually(TIMEOUT, INTERVAL) { + Eventually.eventually(TIMEOUT, patienceInterval) { expectedLogOnCompletion.foreach { e => assert(kubernetesTestComponents.kubernetesClient .pods() @@ -283,21 +348,21 @@ private[spark] class KubernetesSuite extends SparkFunSuite protected def doBasicExecutorPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === image) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === baseMemory) } protected def doBasicExecutorPyPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === pyImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } protected def doBasicExecutorRPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === rImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } @@ -344,6 +409,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite private[spark] object KubernetesSuite { val k8sTestTag = Tag("k8s") val SPARK_PI_MAIN_CLASS: String = "org.apache.spark.examples.SparkPi" + val SPARK_DFS_READ_WRITE_TEST = "org.apache.spark.examples.DFSReadWriteTest" val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 5615d6173eebd..5603a0bb67844 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -21,19 +21,24 @@ import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import io.fabric8.kubernetes.client.DefaultKubernetesClient import org.scalatest.concurrent.Eventually +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.JARS +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI.UI_ENABLED private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { - val namespaceOption = Option(System.getProperty("spark.kubernetes.test.namespace")) + val namespaceOption = Option(System.getProperty(CONFIG_KEY_KUBE_NAMESPACE)) val hasUserSpecifiedNamespace = namespaceOption.isDefined val namespace = namespaceOption.getOrElse(UUID.randomUUID().toString.replaceAll("-", "")) val serviceAccountName = - Option(System.getProperty("spark.kubernetes.test.serviceAccountName")) + Option(System.getProperty(CONFIG_KEY_KUBE_SVC_ACCOUNT)) .getOrElse("default") val kubernetesClient = defaultClient.inNamespace(namespace) val clientConfig = kubernetesClient.getConfiguration @@ -67,6 +72,8 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.app.name", "spark-test-app") .set("spark.ui.enabled", "true") .set("spark.testing", "false") + .set(IS_TESTING.key, "false") + .set(UI_ENABLED.key, "true") .set("spark.kubernetes.submission.waitAppCompletion", "false") .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName) } @@ -83,7 +90,7 @@ private[spark] class SparkAppConf { def get(key: String): String = map.getOrElse(key, "") - def setJars(jars: Seq[String]): Unit = set("spark.jars", jars.mkString(",")) + def setJars(jars: Seq[String]): Unit = set(JARS.key, jars.mkString(",")) override def toString: String = map.toString @@ -120,7 +127,7 @@ private[spark] object SparkAppLauncher extends Logging { appConf.toStringArray :+ appArguments.mainAppResource if (appArguments.appArgs.nonEmpty) { - commandLine += appArguments.appArgs.mkString(" ") + commandLine ++= appArguments.appArgs.to[ArrayBuffer] } logInfo(s"Launching a spark app with command line: ${commandLine.mkString(" ")}") ProcessUtils.executeProcess(commandLine.toArray, timeoutSecs) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala new file mode 100644 index 0000000000000..d7a237f999c02 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import java.io.{File, PrintWriter} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.api.model.storage.StorageClassBuilder +import org.scalatest.Tag +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Milliseconds, Span} + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ + +private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => + import PVTestsSuite._ + + private def setupLocalStorage(): Unit = { + val scBuilder = new StorageClassBuilder() + .withKind("StorageClass") + .withApiVersion("storage.k8s.io/v1") + .withNewMetadata() + .withName(STORAGE_NAME) + .endMetadata() + .withProvisioner("kubernetes.io/no-provisioner") + .withVolumeBindingMode("WaitForFirstConsumer") + + val pvBuilder = new PersistentVolumeBuilder() + .withKind("PersistentVolume") + .withApiVersion("v1") + .withNewMetadata() + .withName("test-local-pv") + .endMetadata() + .withNewSpec() + .withCapacity(Map("storage" -> new QuantityBuilder().withAmount("1Gi").build()).asJava) + .withAccessModes("ReadWriteOnce") + .withPersistentVolumeReclaimPolicy("Retain") + .withStorageClassName("test-local-storage") + .withLocal(new LocalVolumeSourceBuilder().withPath(VM_PATH).build()) + .withNewNodeAffinity() + .withNewRequired() + .withNodeSelectorTerms(new NodeSelectorTermBuilder() + .withMatchExpressions(new NodeSelectorRequirementBuilder() + .withKey("kubernetes.io/hostname") + .withOperator("In") + .withValues("minikube").build()).build()) + .endRequired() + .endNodeAffinity() + .endSpec() + + val pvcBuilder = new PersistentVolumeClaimBuilder() + .withKind("PersistentVolumeClaim") + .withApiVersion("v1") + .withNewMetadata() + .withName(PVC_NAME) + .endMetadata() + .withNewSpec() + .withAccessModes("ReadWriteOnce") + .withStorageClassName("test-local-storage") + .withResources(new ResourceRequirementsBuilder() + .withRequests(Map("storage" -> new QuantityBuilder() + .withAmount("1Gi").build()).asJava).build()) + .endSpec() + + kubernetesTestComponents + .kubernetesClient + .storage() + .storageClasses() + .create(scBuilder.build()) + + kubernetesTestComponents + .kubernetesClient + .persistentVolumes() + .create(pvBuilder.build()) + + kubernetesTestComponents + .kubernetesClient + .persistentVolumeClaims() + .create(pvcBuilder.build()) + } + + private def deleteLocalStorage(): Unit = { + kubernetesTestComponents + .kubernetesClient + .persistentVolumeClaims() + .withName(PVC_NAME) + .delete() + + kubernetesTestComponents + .kubernetesClient + .persistentVolumes() + .withName(PV_NAME) + .delete() + + kubernetesTestComponents + .kubernetesClient + .storage() + .storageClasses() + .withName(STORAGE_NAME) + .delete() + } + + private def checkPVs(pod: Pod, file: String) = { + Eventually.eventually(TIMEOUT, INTERVAL) { + implicit val podName: String = pod.getMetadata.getName + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val contents = Utils.executeCommand("cat", s"$CONTAINER_MOUNT_PATH/$file") + assert(contents.toString.trim.equals(FILE_CONTENTS)) + } + } + + private def createTempFile(): String = { + val filename = try { + val f = File.createTempFile("tmp", ".txt", new File(HOST_PATH)) + f.deleteOnExit() + new PrintWriter(f) { + try { + write(FILE_CONTENTS) + } finally { + close() + } + } + f.getName + } catch { + case e: Exception => e.printStackTrace(); throw e; + } + filename + } + + test("Test PVs with local storage", k8sTestTag, MinikubeTag) { + sparkAppConf + .set(s"spark.kubernetes.driver.volumes.persistentVolumeClaim.data.mount.path", + CONTAINER_MOUNT_PATH) + .set(s"spark.kubernetes.driver.volumes.persistentVolumeClaim.data.options.claimName", + PVC_NAME) + .set(s"spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.path", + CONTAINER_MOUNT_PATH) + .set(s"spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.claimName", + PVC_NAME) + val file = createTempFile() + try { + setupLocalStorage() + runDFSReadWriteAndVerifyCompletion( + FILE_CONTENTS.split(" ").length, + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + checkPVs(driverPod, file) + }, + executorPodChecker = (executorPod: Pod) => { + doBasicExecutorPodCheck(executorPod) + checkPVs(executorPod, file) + }, + appArgs = Array(s"$CONTAINER_MOUNT_PATH/$file", s"$CONTAINER_MOUNT_PATH"), + interval = Some(PV_TESTS_INTERVAL) + ) + } finally { + // make sure this always run + deleteLocalStorage() + } + } +} + +private[spark] object PVTestsSuite { + val MinikubeTag = Tag("minikube") + val STORAGE_NAME = "test-local-storage" + val PV_NAME = "test-local-pv" + val PVC_NAME = "test-local-pvc" + val CONTAINER_MOUNT_PATH = "/opt/spark/pv-tests" + val HOST_PATH = sys.env.getOrElse("PVC_TESTS_HOST_PATH", "/tmp") + val VM_PATH = sys.env.getOrElse("PVC_TESTS_VM_PATH", "/tmp") + val FILE_CONTENTS = "test PVs" + val PV_TESTS_INTERVAL = PatienceConfiguration.Interval(Span(10, Milliseconds)) +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala new file mode 100644 index 0000000000000..e5a847e7210cb --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import java.io.File + +import io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.k8sTestTag + +private[spark] trait PodTemplateSuite { k8sSuite: KubernetesSuite => + + import PodTemplateSuite._ + + test("Start pod creation from template", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.driver.podTemplateFile", DRIVER_TEMPLATE_FILE.getAbsolutePath) + .set("spark.kubernetes.executor.podTemplateFile", EXECUTOR_TEMPLATE_FILE.getAbsolutePath) + runSparkPiAndVerifyCompletion( + driverPodChecker = (driverPod: Pod) => { + assert(driverPod.getMetadata.getName === driverPodName) + assert(driverPod.getSpec.getContainers.get(0).getImage === image) + assert(driverPod.getSpec.getContainers.get(0).getName === "test-driver-container") + assert(driverPod.getMetadata.getLabels.containsKey(LABEL_KEY)) + assert(driverPod.getMetadata.getLabels.get(LABEL_KEY) === "driver-template-label-value") + }, + executorPodChecker = (executorPod: Pod) => { + assert(executorPod.getSpec.getContainers.get(0).getImage === image) + assert(executorPod.getSpec.getContainers.get(0).getName === "test-executor-container") + assert(executorPod.getMetadata.getLabels.containsKey(LABEL_KEY)) + assert(executorPod.getMetadata.getLabels.get(LABEL_KEY) === "executor-template-label-value") + } + ) + } +} + +private[spark] object PodTemplateSuite { + val LABEL_KEY = "template-label-key" + val DRIVER_TEMPLATE_FILE = new File(getClass.getResource("/driver-template.yml").getFile) + val EXECUTOR_TEMPLATE_FILE = new File(getClass.getResource("/executor-template.yml").getFile) +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index d8f3a6cec05c3..9ead70f670891 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -28,7 +28,10 @@ object ProcessUtils extends Logging { * executeProcess is used to run a command and return the output if it * completes within timeout seconds. */ - def executeProcess(fullCommand: Array[String], timeout: Long): Seq[String] = { + def executeProcess( + fullCommand: Array[String], + timeout: Long, + dumpErrors: Boolean = false): Seq[String] = { val pb = new ProcessBuilder().command(fullCommand: _*) pb.redirectErrorStream(true) val proc = pb.start() @@ -40,7 +43,9 @@ object ProcessUtils extends Logging { }) assert(proc.waitFor(timeout, TimeUnit.SECONDS), s"Timed out while executing ${fullCommand.mkString(" ")}") - assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}") + assert(proc.exitValue == 0, + s"Failed to execute ${fullCommand.mkString(" ")}" + + s"${if (dumpErrors) "\n" + outputLines.mkString("\n")}") outputLines } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 06b73107ec236..904279923334f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -16,18 +16,14 @@ */ package org.apache.spark.deploy.k8s.integrationtest -import org.apache.spark.deploy.k8s.integrationtest.TestConfig.{getTestImageRepo, getTestImageTag} - private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => import PythonTestsSuite._ import KubernetesSuite.k8sTestTag - private val pySparkDockerImage = - s"${getTestImageRepo}/spark-py:${getTestImageTag}" test("Run PySpark on simple pi.py example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_PI, mainClass = "", @@ -41,7 +37,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python2 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "2") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -59,7 +55,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python3 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "3") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -77,7 +73,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with memory customization", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.memoryOverheadFactor", s"$memOverheadConstant") .set("spark.executor.pyspark.memory", s"${additionalMemory}m") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala index 885a23cfb4864..e81562a923228 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala @@ -16,16 +16,13 @@ */ package org.apache.spark.deploy.k8s.integrationtest -import org.apache.spark.deploy.k8s.integrationtest.TestConfig.{getTestImageRepo, getTestImageTag} - private[spark] trait RTestsSuite { k8sSuite: KubernetesSuite => import RTestsSuite._ import KubernetesSuite.k8sTestTag test("Run SparkR on simple dataframe.R example", k8sTestTag) { - sparkAppConf - .set("spark.kubernetes.container.image", s"${getTestImageRepo}/spark-r:${getTestImageTag}") + sparkAppConf.set("spark.kubernetes.container.image", rImage) runSparkApplicationAndVerifyCompletion( appResource = SPARK_R_DATAFRAME_TEST, mainClass = "", diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index b18a6aebda497..9de0868e8f419 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -53,6 +53,7 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => .delete() } + // TODO: [SPARK-25291] This test is flaky with regards to memory of executors test("Run SparkPi with env and mount secrets.", k8sTestTag) { createTestSecret() sparkAppConf @@ -83,33 +84,18 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => private def checkSecrets(pod: Pod): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { implicit val podName: String = pod.getMetadata.getName - val env = executeCommand("env") + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val env = Utils.executeCommand("env") assert(env.toString.contains(ENV_SECRET_VALUE_1)) assert(env.toString.contains(ENV_SECRET_VALUE_2)) - val fileUsernameContents = executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") - val filePasswordContents = executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") + val fileUsernameContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") + val filePasswordContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") assert(fileUsernameContents.toString.trim.equals(ENV_SECRET_VALUE_1)) assert(filePasswordContents.toString.trim.equals(ENV_SECRET_VALUE_2)) } } - - private def executeCommand(cmd: String*)(implicit podName: String): String = { - val out = new ByteArrayOutputStream() - val watch = kubernetesTestComponents - .kubernetesClient - .pods() - .withName(podName) - .readingInput(System.in) - .writingOutput(out) - .writingError(System.err) - .withTTY() - .exec(cmd.toArray: _*) - // wait to get some result back - Thread.sleep(1000) - watch.close() - out.flush() - out.toString() - } } private[spark] object SecretsTestsSuite { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala index 8595d0eab1126..2b1fd08164616 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala @@ -17,6 +17,20 @@ package org.apache.spark.deploy.k8s.integrationtest object TestConstants { - val MINIKUBE_TEST_BACKEND = "minikube" - val GCE_TEST_BACKEND = "gce" + val BACKEND_MINIKUBE = "minikube" + val BACKEND_DOCKER_FOR_DESKTOP = "docker-for-desktop" + val BACKEND_CLOUD = "cloud" + + val CONFIG_KEY_DEPLOY_MODE = "spark.kubernetes.test.deployMode" + val CONFIG_KEY_KUBE_CONFIG_CONTEXT = "spark.kubernetes.test.kubeConfigContext" + val CONFIG_KEY_KUBE_MASTER_URL = "spark.kubernetes.test.master" + val CONFIG_KEY_KUBE_NAMESPACE = "spark.kubernetes.test.namespace" + val CONFIG_KEY_KUBE_SVC_ACCOUNT = "spark.kubernetes.test.serviceAccountName" + val CONFIG_KEY_IMAGE_JVM = "spark.kubernetes.test.jvmImage" + val CONFIG_KEY_IMAGE_PYTHON = "spark.kubernetes.test.pythonImage" + val CONFIG_KEY_IMAGE_R = "spark.kubernetes.test.rImage" + val CONFIG_KEY_IMAGE_TAG = "spark.kubernetes.test.imageTag" + val CONFIG_KEY_IMAGE_TAG_FILE = "spark.kubernetes.test.imageTagFile" + val CONFIG_KEY_IMAGE_REPO = "spark.kubernetes.test.imageRepo" + val CONFIG_KEY_UNPACK_DIR = "spark.kubernetes.test.unpackSparkDir" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala index 663f8b6523ac8..d425f707180c8 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.Closeable import java.net.URI +import org.apache.commons.io.output.ByteArrayOutputStream + import org.apache.spark.internal.Logging object Utils extends Logging { @@ -27,4 +29,24 @@ object Utils extends Logging { val resource = createResource try f.apply(resource) finally resource.close() } + + def executeCommand(cmd: String*)( + implicit podName: String, + kubernetesTestComponents: KubernetesTestComponents): String = { + val out = new ByteArrayOutputStream() + val watch = kubernetesTestComponents + .kubernetesClient + .pods() + .withName(podName) + .readingInput(System.in) + .writingOutput(out) + .writingError(System.err) + .withTTY() + .exec(cmd.toArray: _*) + // wait to get some result back + Thread.sleep(1000) + watch.close() + out.flush() + out.toString() + } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index 284712c6d250e..56ddae0c9c57c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -19,6 +19,9 @@ package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ +import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend +import org.apache.spark.deploy.k8s.integrationtest.backend.docker.DockerForDesktopBackend import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend private[spark] trait IntegrationTestBackend { @@ -28,16 +31,16 @@ private[spark] trait IntegrationTestBackend { } private[spark] object IntegrationTestBackendFactory { - val deployModeConfigKey = "spark.kubernetes.test.deployMode" - def getTestBackend: IntegrationTestBackend = { - val deployMode = Option(System.getProperty(deployModeConfigKey)) - .getOrElse("minikube") - if (deployMode == "minikube") { - MinikubeTestBackend - } else { - throw new IllegalArgumentException( - "Invalid " + deployModeConfigKey + ": " + deployMode) + val deployMode = Option(System.getProperty(CONFIG_KEY_DEPLOY_MODE)) + .getOrElse(BACKEND_MINIKUBE) + deployMode match { + case BACKEND_MINIKUBE => MinikubeTestBackend + case BACKEND_CLOUD => + new KubeConfigBackend(System.getProperty(CONFIG_KEY_KUBE_CONFIG_CONTEXT)) + case BACKEND_DOCKER_FOR_DESKTOP => DockerForDesktopBackend + case _ => throw new IllegalArgumentException("Invalid " + + CONFIG_KEY_DEPLOY_MODE + ": " + deployMode) } } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala new file mode 100644 index 0000000000000..be1834c0b5dea --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest.backend.cloud + +import java.nio.file.Paths + +import io.fabric8.kubernetes.client.{Config, DefaultKubernetesClient} +import io.fabric8.kubernetes.client.utils.Utils +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.deploy.k8s.integrationtest.TestConstants +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils.checkAndGetK8sMasterUrl + +private[spark] class KubeConfigBackend(var context: String) + extends IntegrationTestBackend with Logging { + logInfo(s"K8S Integration tests will run against " + + s"${if (context != null) s"context ${context}" else "default context"}" + + s" from users K8S config file") + + private var defaultClient: DefaultKubernetesClient = _ + + override def initialize(): Unit = { + // Auto-configure K8S client from K8S config file + if (Utils.getSystemPropertyOrEnvVar(Config.KUBERNETES_KUBECONFIG_FILE, null: String) == null) { + // Fabric 8 client will automatically assume a default location in this case + logWarning("No explicit KUBECONFIG specified, will assume $HOME/.kube/config") + } + val config = Config.autoConfigure(context) + + // If an explicit master URL was specified then override that detected from the + // K8S config if it is different + var masterUrl = Option(System.getProperty(TestConstants.CONFIG_KEY_KUBE_MASTER_URL)) + .getOrElse(null) + if (StringUtils.isNotBlank(masterUrl)) { + // Clean up master URL which would have been specified in Spark format into a normal + // K8S master URL + masterUrl = checkAndGetK8sMasterUrl(masterUrl).replaceFirst("k8s://", "") + if (!StringUtils.equals(config.getMasterUrl, masterUrl)) { + logInfo(s"Overriding K8S master URL ${config.getMasterUrl} from K8S config file " + + s"with user specified master URL ${masterUrl}") + config.setMasterUrl(masterUrl) + } + } + + defaultClient = new DefaultKubernetesClient(config) + } + + override def cleanUp(): Unit = { + super.cleanUp() + } + + override def getKubernetesClient: DefaultKubernetesClient = { + defaultClient + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala similarity index 68% rename from core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala index af67cbbce4e51..81a11ae9dcdc6 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala @@ -14,17 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.deploy.k8s.integrationtest.backend.docker -package org.apache.spark.api.r +import org.apache.spark.deploy.k8s.integrationtest.TestConstants +import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend -private[spark] object SparkRDefaults { +private[spark] object DockerForDesktopBackend + extends KubeConfigBackend(TestConstants.BACKEND_DOCKER_FOR_DESKTOP) { - // Default value for spark.r.backendConnectionTimeout config - val DEFAULT_CONNECTION_TIMEOUT: Int = 6000 - - // Default value for spark.r.heartBeatInterval config - val DEFAULT_HEARTBEAT_INTERVAL: Int = 100 - - // Default value for spark.r.numRBackendThreads config - val DEFAULT_NUM_RBACKEND_THREADS = 2 } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala index cb9324179d70e..f92977ddacdf5 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -25,6 +25,7 @@ private[spark] object MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ override def initialize(): Unit = { + Minikube.logVersion() val minikubeStatus = Minikube.getMinikubeStatus require(minikubeStatus == MinikubeStatus.RUNNING, s"Minikube must be running to use the Minikube backend for integration tests." + diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 64698b55c6bb6..f0117147a966c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -24,6 +24,7 @@ import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, SparkUncaughtExceptionHandler, Utils} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index d134847dc74d2..2b8655ceee0cd 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -63,11 +63,6 @@ package object config { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("30s") - private[spark] val RECOVERY_MODE = - ConfigBuilder("spark.deploy.recoveryMode") - .stringConf - .createWithDefault("NONE") - private[spark] val DISPATCHER_WEBUI_URL = ConfigBuilder("spark.mesos.dispatcher.webui.url") .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + @@ -75,13 +70,6 @@ package object config { .stringConf .createOptional - private[spark] val ZOOKEEPER_URL = - ConfigBuilder("spark.deploy.zookeeper.url") - .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + - "configuration is used to set the zookeeper URL to connect to.") - .stringConf - .createOptional - private[spark] val HISTORY_SERVER_URL = ConfigBuilder("spark.mesos.dispatcher.historyServer.url") .doc("Set the URL of the history server. The dispatcher will then " + @@ -129,4 +117,7 @@ package object config { "when launching drivers. Default is to accept all offers with sufficient resources.") .stringConf .createWithDefault("") + + private[spark] val EXECUTOR_URI = + ConfigBuilder("spark.executor.uri").stringConf.createOptional } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 68f6921153d89..a4aba3e9c0d05 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -27,6 +27,7 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest._ +import org.apache.spark.internal.config import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils @@ -92,12 +93,12 @@ private[mesos] class MesosSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) val name = request.sparkProperties.getOrElse("spark.app.name", mainClass) // Construct driver description diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 61ab3e87c5711..123412f21e2a2 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -26,6 +26,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.util.Utils /** @@ -94,13 +95,13 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( zk: CuratorFramework, conf: SparkConf) extends MesosClusterPersistenceEngine with Logging { - private val WORKING_DIR = - conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir + private val workingDir = + conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark_mesos_dispatcher") + "/" + baseDir - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) def path(name: String): String = { - WORKING_DIR + "/" + name + workingDir + "/" + name } override def expunge(name: String): Unit = { @@ -129,6 +130,6 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( } override def fetchAll[T](): Iterable[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T]) + zk.getChildren.forPath(workingDir).asScala.flatMap(fetch[T]) } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 8c9a6b44ec6ae..01a294478a5f9 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -32,6 +32,7 @@ import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} +import org.apache.spark.internal.config._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils @@ -365,8 +366,7 @@ private[spark] class MesosClusterScheduler( } private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = { - desc.conf.getOption("spark.executor.uri") - .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) + desc.conf.get(config.EXECUTOR_URI).orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) } private def getDriverFrameworkID(desc: MesosDriverDescription): String = { @@ -432,7 +432,7 @@ private[spark] class MesosClusterScheduler( private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = { val confUris = List(conf.getOption("spark.mesos.uris"), desc.conf.getOption("spark.mesos.uris"), - desc.conf.getOption("spark.submit.pyFiles")).flatMap( + Some(desc.conf.get(SUBMIT_PYTHON_FILES).mkString(","))).flatMap( _.map(_.split(",").map(_.trim)) ).flatten @@ -474,7 +474,7 @@ private[spark] class MesosClusterScheduler( } else if (executorUri.isDefined) { val folderBasename = executorUri.get.split('/').last.split('.').head - val entries = conf.getOption("spark.executor.extraLibraryPath") + val entries = conf.get(EXECUTOR_LIBRARY_PATH) .map(path => Seq(path) ++ desc.command.libraryPathEntries) .getOrElse(desc.command.libraryPathEntries) @@ -528,22 +528,22 @@ private[spark] class MesosClusterScheduler( options ++= Seq("--class", desc.command.mainClass) } - desc.conf.getOption("spark.executor.memory").foreach { v => + desc.conf.getOption(EXECUTOR_MEMORY.key).foreach { v => options ++= Seq("--executor-memory", v) } - desc.conf.getOption("spark.cores.max").foreach { v => + desc.conf.getOption(CORES_MAX.key).foreach { v => options ++= Seq("--total-executor-cores", v) } - desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val formattedFiles = pyFiles.split(",") - .map { path => new File(sandboxPath, path.split("/").last).toString() } - .mkString(",") - options ++= Seq("--py-files", formattedFiles) - } + + val pyFiles = desc.conf.get(SUBMIT_PYTHON_FILES) + val formattedFiles = pyFiles.map { path => + new File(sandboxPath, path.split("/").last).toString() + }.mkString(",") + options ++= Seq("--py-files", formattedFiles) // --conf val replicatedOptionsBlacklist = Set( - "spark.jars", // Avoids duplicate classes in classpath + JARS.key, // Avoids duplicate classes in classpath "spark.submit.deployMode", // this would be set to `cluster`, but we need client "spark.master" // this contains the address of the dispatcher, not master ) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 178de30f0f381..d0174516c2361 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -26,12 +26,12 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future -import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException, TaskState} import org.apache.spark.deploy.mesos.config._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf @@ -59,15 +59,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler with MesosSchedulerUtils { - private lazy val hadoopDelegationTokenManager: MesosHadoopDelegationTokenManager = - new MesosHadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint) - // Blacklist a slave after this many failures private val MAX_SLAVE_FAILURES = 2 - private val maxCoresOption = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCoresOption = conf.get(config.CORES_MAX) - private val executorCoresOption = conf.getOption("spark.executor.cores").map(_.toInt) + private val executorCoresOption = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) private val minCoresPerExecutor = executorCoresOption.getOrElse(1) @@ -222,18 +219,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { val environment = Environment.newBuilder() - val extraClassPath = conf.getOption("spark.executor.extraClassPath") + val extraClassPath = conf.get(config.EXECUTOR_CLASS_PATH) extraClassPath.foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, taskId) }.getOrElse("") // Set the environment variable through a command prefix // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -263,8 +260,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val uri = conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) if (uri.isEmpty) { val executorSparkHome = conf.getOption("spark.mesos.executor.home") @@ -306,8 +302,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( "driverURL" } else { RpcEndpointAddress( - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, + conf.get(config.DRIVER_HOST_ADDRESS), + conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString } } @@ -635,7 +631,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( externalShufflePort, sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s")}s"), - sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)) slave.shuffleRegistered = true } @@ -677,7 +673,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( launcherBackend.close() } - private def stopSchedulerBackend() { + private def stopSchedulerBackend(): Unit = { // Make sure we're not launching tasks during shutdown stateLock.synchronized { if (stopCalled) { @@ -776,6 +772,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) + } + private def numExecutors(): Int = { slaves.values.map(_.taskIDs.size).sum } @@ -788,14 +788,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( offer.getHostname } } - - override def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { - if (UserGroupInformation.isSecurityEnabled) { - Some(hadoopDelegationTokenManager.getTokens()) - } else { - None - } - } } private class Slave(val hostname: String) { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index 0bb6fe0fa4bdf..192f9407a1ba4 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -28,8 +28,9 @@ import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString import org.apache.spark.{SparkContext, SparkException, TaskState} -import org.apache.spark.deploy.mesos.config +import org.apache.spark.deploy.mesos.config.EXECUTOR_URI import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.internal.config import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -107,15 +108,15 @@ private[spark] class MesosFineGrainedSchedulerBackend( throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => + sc.conf.get(config.EXECUTOR_CLASS_PATH).foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, execId) }.getOrElse("") - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = sc.conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -132,8 +133,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val uri = sc.conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) val executorBackendName = classOf[MesosExecutorBackend].getName if (uri.isEmpty) { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala deleted file mode 100644 index a1bf4f0c048fe..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.ThreadUtils - - -/** - * The MesosHadoopDelegationTokenManager fetches and updates Hadoop delegation tokens on the behalf - * of the MesosCoarseGrainedSchedulerBackend. It is modeled after the YARN AMCredentialRenewer, - * and similarly will renew the Credentials when 75% of the renewal interval has passed. - * The principal difference is that instead of writing the new credentials to HDFS and - * incrementing the timestamp of the file, the new credentials (called Tokens when they are - * serialized) are broadcast to all running executors. On the executor side, when new Tokens are - * received they overwrite the current credentials. - */ -private[spark] class MesosHadoopDelegationTokenManager( - conf: SparkConf, - hadoopConfig: Configuration, - driverEndpoint: RpcEndpointRef) - extends Logging { - - require(driverEndpoint != null, "DriverEndpoint is not initialized") - - private val credentialRenewerThread: ScheduledExecutorService = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") - - private val tokenManager: HadoopDelegationTokenManager = - new HadoopDelegationTokenManager(conf, hadoopConfig) - - private val principal: String = conf.get(config.PRINCIPAL).orNull - - private var (tokens: Array[Byte], timeOfNextRenewal: Long) = { - try { - val creds = UserGroupInformation.getCurrentUser.getCredentials - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) - logInfo(s"Initialized tokens: ${SparkHadoopUtil.get.dumpTokens(creds)}") - (SparkHadoopUtil.get.serialize(creds), SparkHadoopUtil.nextCredentialRenewalTime(rt, conf)) - } catch { - case e: Exception => - logError(s"Failed to fetch Hadoop delegation tokens $e") - throw e - } - } - - private val keytabFile: Option[String] = conf.get(config.KEYTAB) - - scheduleTokenRenewal() - - private def scheduleTokenRenewal(): Unit = { - if (keytabFile.isDefined) { - require(principal != null, "Principal is required for Keytab-based authentication") - logInfo(s"Using keytab: ${keytabFile.get} and principal $principal") - } else { - logInfo("Using ticket cache for Kerberos authentication, no token renewal.") - return - } - - def scheduleRenewal(runnable: Runnable): Unit = { - val remainingTime = timeOfNextRenewal - System.currentTimeMillis() - if (remainingTime <= 0) { - logInfo("Credentials have expired, creating new ones now.") - runnable.run() - } else { - logInfo(s"Scheduling login from keytab in $remainingTime millis.") - credentialRenewerThread.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS) - } - } - - val credentialRenewerRunnable = - new Runnable { - override def run(): Unit = { - try { - getNewDelegationTokens() - broadcastDelegationTokens(tokens) - } catch { - case e: Exception => - // Log the error and try to write new tokens back in an hour - val delay = TimeUnit.SECONDS.toMillis(conf.get(config.CREDENTIALS_RENEWAL_RETRY_WAIT)) - logWarning( - s"Couldn't broadcast tokens, trying again in ${UIUtils.formatDuration(delay)}", e) - credentialRenewerThread.schedule(this, delay, TimeUnit.MILLISECONDS) - return - } - scheduleRenewal(this) - } - } - scheduleRenewal(credentialRenewerRunnable) - } - - private def getNewDelegationTokens(): Unit = { - logInfo(s"Attempting to login to KDC with principal ${principal}") - // Get new delegation tokens by logging in with a new UGI inspired by AMCredentialRenewer.scala - // Don't protect against keytabFile being empty because it's guarded above. - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabFile.get) - logInfo("Successfully logged into KDC") - val tempCreds = ugi.getCredentials - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val nextRenewalTime = ugi.doAs(new PrivilegedExceptionAction[Long] { - override def run(): Long = { - tokenManager.obtainDelegationTokens(hadoopConf, tempCreds) - } - }) - - val currTime = System.currentTimeMillis() - timeOfNextRenewal = if (nextRenewalTime <= currTime) { - logWarning(s"Next credential renewal time ($nextRenewalTime) is earlier than " + - s"current time ($currTime), which is unexpected, please check your credential renewal " + - "related configurations in the target services.") - currTime - } else { - SparkHadoopUtil.nextCredentialRenewalTime(nextRenewalTime, conf) - } - logInfo(s"Time of next renewal is in ${timeOfNextRenewal - System.currentTimeMillis()} ms") - - // Add the temp credentials back to the original ones. - UserGroupInformation.getCurrentUser.addCredentials(tempCreds) - // update tokens for late or dynamically added executors - tokens = SparkHadoopUtil.get.serialize(tempCreds) - } - - private def broadcastDelegationTokens(tokens: Array[Byte]) = { - logInfo("Sending new tokens to all executors") - driverEndpoint.send(UpdateDelegationTokens(tokens)) - } - - def getTokens(): Array[Byte] = { - tokens - } -} - diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 8ef1e18f83de3..634460686bb2b 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -37,7 +37,7 @@ import org.apache.mesos.protobuf.{ByteString, GeneratedMessageV3} import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.TaskState import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.{Status => _, _} import org.apache.spark.util.Utils /** diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5ff826a2e1181..5c0dc47d46951 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -41,9 +41,10 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.AMCredentialRenewer +import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} @@ -99,20 +100,18 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private val credentialRenewer: Option[AMCredentialRenewer] = sparkConf.get(KEYTAB).map { _ => - new AMCredentialRenewer(sparkConf, yarnConf) + private val tokenManager: Option[YARNHadoopDelegationTokenManager] = { + sparkConf.get(KEYTAB).map { _ => + new YARNHadoopDelegationTokenManager(sparkConf, yarnConf) + } } - private val ugi = credentialRenewer match { - case Some(cr) => + private val ugi = tokenManager match { + case Some(tm) => // Set the context class loader so that the token renewer has access to jars distributed // by the user. - val currentLoader = Thread.currentThread().getContextClassLoader() - Thread.currentThread().setContextClassLoader(userClassLoader) - try { - cr.start() - } finally { - Thread.currentThread().setContextClassLoader(currentLoader) + Utils.withContextClassLoader(userClassLoader) { + tm.start() } case _ => @@ -256,7 +255,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends if (isClusterMode) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") + System.setProperty(UI_PORT.key, "0") // Set the master and deploy mode property to match the requested mode. System.setProperty("spark.master", "yarn") @@ -380,7 +379,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends userClassThread.interrupt() } if (!inShutdown) { - credentialRenewer.foreach(_.stop()) + tokenManager.foreach(_.stop()) } } } @@ -440,7 +439,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends securityMgr, localResources) - credentialRenewer.foreach(_.setDriverRef(driverRef)) + tokenManager.foreach(_.setDriverRef(driverRef)) // Initialize the AM endpoint *after* the allocator has been initialized. This ensures // that when the driver sends an initial executor request (e.g. after an AM restart), @@ -471,8 +470,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends rpcEnv = sc.env.rpcEnv val userConf = sc.getConf - val host = userConf.get("spark.driver.host") - val port = userConf.get("spark.driver.port").toInt + val host = userConf.get(DRIVER_HOST_ADDRESS) + val port = userConf.get(DRIVER_PORT) registerAM(host, port, userConf, sc.ui.map(_.webUrl)) val driverRef = rpcEnv.setupEndpointRef( @@ -506,7 +505,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends amCores, true) // The client-mode AM doesn't listen for incoming connections, so report an invalid port. - registerAM(hostname, -1, sparkConf, sparkConf.getOption("spark.driver.appUIAddress")) + registerAM(hostname, -1, sparkConf, sparkConf.get(DRIVER_APP_UI_ADDRESS)) // The driver should be up and listening, so unlike cluster mode, just try to connect to it // with no waiting or retrying. @@ -646,7 +645,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends d.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) case None => - System.setProperty("spark.ui.filters", amFilter) + System.setProperty(UI_FILTERS.key, amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 4a85898ef880b..0034c04700d27 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -53,6 +53,7 @@ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} import org.apache.spark.util.{CallerContext, Utils} @@ -67,7 +68,7 @@ private[spark] class Client( private val yarnClient = YarnClient.createYarnClient private val hadoopConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) - private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster" + private val isClusterMode = sparkConf.get(SUBMIT_DEPLOY_MODE) == "cluster" // AM related configurations private val amMemory = if (isClusterMode) { @@ -154,6 +155,8 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { + ResourceRequestHelper.validateResources(sparkConf) + var appId: ApplicationId = null try { launcherBackend.connect() @@ -234,6 +237,13 @@ private[spark] class Client( def createApplicationSubmissionContext( newApp: YarnClientApplication, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { + val amResources = + if (isClusterMode) { + sparkConf.getAllWithPrefix(config.YARN_DRIVER_RESOURCE_TYPES_PREFIX).toMap + } else { + sparkConf.getAllWithPrefix(config.YARN_AM_RESOURCE_TYPES_PREFIX).toMap + } + logDebug(s"AM resources: $amResources") val appContext = newApp.getApplicationSubmissionContext appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark")) appContext.setQueue(sparkConf.get(QUEUE_NAME)) @@ -256,6 +266,10 @@ private[spark] class Client( val capability = Records.newRecord(classOf[Resource]) capability.setMemory(amMemory + amMemoryOverhead) capability.setVirtualCores(amCores) + if (amResources.nonEmpty) { + ResourceRequestHelper.setResourceRequests(amResources, capability) + } + logDebug(s"Created resource capability for AM request: $capability") sparkConf.get(AM_NODE_LABEL_EXPRESSION) match { case Some(expr) => @@ -306,7 +320,7 @@ private[spark] class Client( private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val credentials = UserGroupInformation.getCurrentUser().getCredentials() val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - credentialManager.obtainDelegationTokens(hadoopConf, credentials) + credentialManager.obtainDelegationTokens(credentials) // When using a proxy user, copy the delegation tokens to the user's credentials. Avoid // that for regular users, since in those case the user already has access to the TGT, @@ -1520,8 +1534,8 @@ private[spark] class YarnClusterApplication extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, // so remove them from sparkConf here for yarn mode. - conf.remove("spark.jars") - conf.remove("spark.files") + conf.remove(JARS) + conf.remove(FILES) new Client(new ClientArguments(args), conf).run() } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala new file mode 100644 index 0000000000000..012268ea856f5 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.lang.{Long => JLong} +import java.lang.reflect.InvocationTargetException + +import scala.collection.mutable +import scala.util.Try + +import org.apache.hadoop.yarn.api.records.Resource + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.{CausedBy, Utils} + +/** + * This helper class uses some of Hadoop 3 methods from the YARN API, + * so we need to use reflection to avoid compile error when building against Hadoop 2.x + */ +private object ResourceRequestHelper extends Logging { + private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r + private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" + + /** + * Validates sparkConf and throws a SparkException if any of standard resources (memory or cores) + * is defined with the property spark.yarn.x.resource.y + * Need to reject all combinations of AM / Driver / Executor and memory / CPU cores resources, as + * Spark has its own names for them (memory, cores), + * but YARN have its names too: (memory, memory-mb, mb) and (cores, vcores, cpu-vcores). + * We need to disable every possible way YARN could receive the resource definitions above. + */ + def validateResources(sparkConf: SparkConf): Unit = { + val resourceDefinitions = Seq[(String, String)]( + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory"), + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "mb"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "mb"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb"), + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory-mb"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory-mb"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cores"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "vcores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "vcores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cpu-vcores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cpu-vcores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cpu-vcores")) + val errorMessage = new mutable.StringBuilder() + + resourceDefinitions.foreach { case (sparkName, resourceRequest) => + if (sparkConf.contains(resourceRequest)) { + errorMessage.append(s"Error: Do not use $resourceRequest, " + + s"please use $sparkName instead!\n") + } + } + + if (errorMessage.nonEmpty) { + throw new SparkException(errorMessage.toString()) + } + } + + /** + * Sets resource amount with the corresponding unit to the passed resource object. + * @param resources resource values to set + * @param resource resource object to update + */ + def setResourceRequests( + resources: Map[String, String], + resource: Resource): Unit = { + require(resource != null, "Resource parameter should not be null!") + + logDebug(s"Custom resources requested: $resources") + if (!isYarnResourceTypesAvailable()) { + if (resources.nonEmpty) { + logWarning("Ignoring custom resource requests because " + + "the version of YARN does not support it!") + } + return + } + + val resInfoClass = Utils.classForName(RESOURCE_INFO_CLASS) + val setResourceInformationMethod = + resource.getClass.getMethod("setResourceInformation", classOf[String], resInfoClass) + resources.foreach { case (name, rawAmount) => + try { + val AMOUNT_AND_UNIT_REGEX(amountPart, unitPart) = rawAmount + val amount = amountPart.toLong + val unit = unitPart match { + case "g" => "G" + case "t" => "T" + case "p" => "P" + case _ => unitPart + } + logDebug(s"Registering resource with name: $name, amount: $amount, unit: $unit") + val resourceInformation = createResourceInformation(name, amount, unit, resInfoClass) + setResourceInformationMethod.invoke( + resource, name, resourceInformation.asInstanceOf[AnyRef]) + } catch { + case _: MatchError => + throw new IllegalArgumentException(s"Resource request for '$name' ('$rawAmount') " + + s"does not match pattern $AMOUNT_AND_UNIT_REGEX.") + case CausedBy(e: IllegalArgumentException) => + throw new IllegalArgumentException(s"Invalid request for $name: ${e.getMessage}") + case e: InvocationTargetException if e.getCause != null => throw e.getCause + } + } + } + + private def createResourceInformation( + resourceName: String, + amount: Long, + unit: String, + resInfoClass: Class[_]): Any = { + val resourceInformation = + if (unit.nonEmpty) { + val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", + classOf[String], classOf[String], JLong.TYPE) + resInfoNewInstanceMethod.invoke(null, resourceName, unit, amount.asInstanceOf[JLong]) + } else { + val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", + classOf[String], JLong.TYPE) + resInfoNewInstanceMethod.invoke(null, resourceName, amount.asInstanceOf[JLong]) + } + resourceInformation + } + + /** + * Checks whether Hadoop 2.x or 3 is used as a dependency. + * In case of Hadoop 3 and later, the ResourceInformation class + * should be available on the classpath. + */ + def isYarnResourceTypesAvailable(): Boolean = { + Try(Utils.classForName(RESOURCE_INFO_CLASS)).isSuccess + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 96bc1c7889500..c2e8451303ba3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -37,6 +37,7 @@ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor @@ -140,10 +141,18 @@ private[yarn] class YarnAllocator( } // Number of cores per executor. protected val executorCores = sparkConf.get(EXECUTOR_CORES) - // Resource capability requested for each executors - private[yarn] val resource = Resource.newInstance( - executorMemory + memoryOverhead + pysparkWorkerMemory, - executorCores) + + private val executorResourceRequests = + sparkConf.getAllWithPrefix(config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX).toMap + + // Resource capability requested for each executor + private[yarn] val resource: Resource = { + val resource = Resource.newInstance( + executorMemory + memoryOverhead + pysparkWorkerMemory, executorCores) + ResourceRequestHelper.setResourceRequests(executorResourceRequests, resource) + logDebug(s"Created resource capability: $resource") + resource + } private val launcherPool = ThreadUtils.newDaemonCachedThreadPool( "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS)) @@ -297,9 +306,16 @@ private[yarn] class YarnAllocator( hostToLocalTaskCounts, pendingAllocate) if (missing > 0) { - logInfo(s"Will request $missing executor container(s), each with " + - s"${resource.getVirtualCores} core(s) and " + - s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)") + if (log.isInfoEnabled()) { + var requestContainerMessage = s"Will request $missing executor container(s), each with " + + s"${resource.getVirtualCores} core(s) and " + + s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)" + if (ResourceRequestHelper.isYarnResourceTypesAvailable() && + executorResourceRequests.nonEmpty) { + requestContainerMessage ++= s" with custom resources: " + resource.toString + } + logInfo(requestContainerMessage) + } // cancel "stale" requests for locations that are no longer needed staleRequests.foreach { stale => @@ -486,13 +502,20 @@ private[yarn] class YarnAllocator( // memory, but use the asked vcore count for matching, effectively disabling matching on vcore // count. val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, - resource.getVirtualCores) + resource.getVirtualCores) + + ResourceRequestHelper.setResourceRequests(executorResourceRequests, matchingResource) + + logDebug(s"Calling amClient.getMatchingRequests with parameters: " + + s"priority: ${allocatedContainer.getPriority}, " + + s"location: $location, resource: $matchingResource") val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, matchingResource) // Match the allocation to a request if (!matchingRequests.isEmpty) { val containerRequest = matchingRequests.get(0).iterator.next + logDebug(s"Removing container request via AM client: $containerRequest") amClient.removeContainerRequest(containerRequest) containersToUse += allocatedContainer } else { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index ab8273bd6321d..7e9cd409daf36 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -224,16 +224,12 @@ package object config { /* Driver configuration. */ - private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") - .intConf - .createWithDefault(1) + private[spark] val DRIVER_APP_UI_ADDRESS = ConfigBuilder("spark.driver.appUIAddress") + .stringConf + .createOptional /* Executor configuration. */ - private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores") - .intConf - .createWithDefault(1) - private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.executor.nodeLabelExpression") .doc("Node label expression for executors.") @@ -325,10 +321,6 @@ package object config { .stringConf .createOptional - private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.yarn.kerberos.relogin.period") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("1m") - // The list of cache-related config entries. This is used by Client and the AM to clean // up the environment so that these settings do not appear on the web UI. private[yarn] val CACHE_CONFIGS = Seq( @@ -345,4 +337,8 @@ package object config { .booleanConf .createWithDefault(false) + private[yarn] val YARN_EXECUTOR_RESOURCE_TYPES_PREFIX = "spark.yarn.executor.resource." + private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." + private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." + } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala deleted file mode 100644 index 51ef7d3a60362..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.yarn.security - -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicReference - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.ThreadUtils - -/** - * A manager tasked with periodically updating delegation tokens needed by the application. - * - * This manager is meant to make sure long-running apps (such as Spark Streaming apps) can run - * without interruption while accessing secured services. It periodically logs in to the KDC with - * user-provided credentials, and contacts all the configured secure services to obtain delegation - * tokens to be distributed to the rest of the application. - * - * This class will manage the kerberos login, by renewing the TGT when needed. Because the UGI API - * does not expose the TTL of the TGT, a configuration controls how often to check that a relogin is - * necessary. This is done reasonably often since the check is a no-op when the relogin is not yet - * needed. The check period can be overridden in the configuration. - * - * New delegation tokens are created once 75% of the renewal interval of the original tokens has - * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. - * The driver is tasked with distributing the tokens to other processes that might need them. - */ -private[yarn] class AMCredentialRenewer( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { - - private val principal = sparkConf.get(PRINCIPAL).get - private val keytab = sparkConf.get(KEYTAB).get - private val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - - private val renewalExecutor: ScheduledExecutorService = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Refresh Thread") - - private val driverRef = new AtomicReference[RpcEndpointRef]() - - private val renewalTask = new Runnable() { - override def run(): Unit = { - updateTokensTask() - } - } - - def setDriverRef(ref: RpcEndpointRef): Unit = { - driverRef.set(ref) - } - - /** - * Start the token renewer. Upon start, the renewer will: - * - * - log in the configured user, and set up a task to keep that user's ticket renewed - * - obtain delegation tokens from all available providers - * - schedule a periodic task to update the tokens when needed. - * - * @return The newly logged in user. - */ - def start(): UserGroupInformation = { - val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() - val ugi = doLogin() - - ugi.doAs(new PrivilegedExceptionAction[Unit]() { - override def run(): Unit = { - startInternal(ugi, originalCreds) - } - }) - - ugi - } - - private def startInternal(ugi: UserGroupInformation, originalCreds: Credentials): Unit = { - val tgtRenewalTask = new Runnable() { - override def run(): Unit = { - ugi.checkTGTAndReloginFromKeytab() - } - } - val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) - renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, - TimeUnit.SECONDS) - - val creds = obtainTokensAndScheduleRenewal(ugi) - ugi.addCredentials(creds) - - // Transfer the original user's tokens to the new user, since that's needed to connect to - // YARN. Explicitly avoid overwriting tokens that already exist in the current user's - // credentials, since those were freshly obtained above (see SPARK-23361). - val existing = ugi.getCredentials() - existing.mergeAll(originalCreds) - ugi.addCredentials(existing) - } - - def stop(): Unit = { - renewalExecutor.shutdown() - } - - private def scheduleRenewal(delay: Long): Unit = { - val _delay = math.max(0, delay) - logInfo(s"Scheduling login from keytab in ${UIUtils.formatDuration(delay)}.") - renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) - } - - /** - * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself - * to fetch the next set of tokens when needed. - */ - private def updateTokensTask(): Unit = { - try { - val freshUGI = doLogin() - val creds = obtainTokensAndScheduleRenewal(freshUGI) - val tokens = SparkHadoopUtil.get.serialize(creds) - - val driver = driverRef.get() - if (driver != null) { - logInfo("Updating delegation tokens.") - driver.send(UpdateDelegationTokens(tokens)) - } else { - // This shouldn't really happen, since the driver should register way before tokens expire - // (or the AM should time out the application). - logWarning("Delegation tokens close to expiration but no driver has registered yet.") - } - SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) - } catch { - case e: Exception => - val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) - logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + - " If this happens too often tasks will fail.", e) - scheduleRenewal(delay) - } - } - - /** - * Obtain new delegation tokens from the available providers. Schedules a new task to fetch - * new tokens before the new set expires. - * - * @return Credentials containing the new tokens. - */ - private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { - ugi.doAs(new PrivilegedExceptionAction[Credentials]() { - override def run(): Credentials = { - val creds = new Credentials() - val nextRenewal = credentialManager.obtainDelegationTokens(hadoopConf, creds) - - val timeToWait = SparkHadoopUtil.nextCredentialRenewalTime(nextRenewal, sparkConf) - - System.currentTimeMillis() - scheduleRenewal(timeToWait) - creds - } - }) - } - - private def doLogin(): UserGroupInformation = { - logInfo(s"Attempting to login to KDC using principal: $principal") - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - logInfo("Successfully logged into KDC.") - ugi - } - -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala index 26a2e5d730218..2d9a3f0c83fd2 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala @@ -22,12 +22,13 @@ import java.util.ServiceLoader import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil -import org.apache.spark.internal.Logging +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils /** @@ -36,27 +37,25 @@ import org.apache.spark.util.Utils * in [[HadoopDelegationTokenManager]]. */ private[yarn] class YARNHadoopDelegationTokenManager( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { + _sparkConf: SparkConf, + _hadoopConf: Configuration) + extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { - private val delegationTokenManager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, - conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) - - // public for testing - val credentialProviders = getCredentialProviders + private val credentialProviders = { + ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) + .asScala + .toList + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap + } if (credentialProviders.nonEmpty) { logDebug("Using the following YARN-specific credential providers: " + s"${credentialProviders.keys.mkString(", ")}.") } - /** - * Writes delegation tokens to creds. Delegation tokens are fetched from all registered - * providers. - * - * @return Time after which the fetched delegation tokens should be renewed. - */ - def obtainDelegationTokens(hadoopConf: Configuration, creds: Credentials): Long = { - val superInterval = delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) + override def obtainDelegationTokens(creds: Credentials): Long = { + val superInterval = super.obtainDelegationTokens(creds) credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { @@ -69,18 +68,13 @@ private[yarn] class YARNHadoopDelegationTokenManager( }.foldLeft(superInterval)(math.min) } - private def getCredentialProviders: Map[String, ServiceCredentialProvider] = { - val providers = loadCredentialProviders - - providers. - filter { p => delegationTokenManager.isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap + // For testing. + override def isProviderLoaded(serviceName: String): Boolean = { + credentialProviders.contains(serviceName) || super.isProviderLoaded(serviceName) } - private def loadCredentialProviders: List[ServiceCredentialProvider] = { - ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) - .asScala - .toList + override protected def fileSystemsToAccess(): Set[FileSystem] = { + YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, hadoopConf) } + } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 9397a1e3de9ac..167eef19ed856 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnAppReport} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl @@ -42,10 +42,10 @@ private[spark] class YarnClientSchedulerBackend( * This waits until the application is running. */ override def start() { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") + val driverHost = conf.get(config.DRIVER_HOST_ADDRESS) + val driverPort = conf.get(config.DRIVER_PORT) val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) } + sc.ui.foreach { ui => conf.set(DRIVER_APP_UI_ADDRESS, ui.webUrl) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 63bea3e7a5003..b777e0cdefbae 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -17,19 +17,21 @@ package org.apache.spark.scheduler.cluster -import java.util.concurrent.atomic.{AtomicBoolean} +import java.util +import java.util.concurrent.atomic.AtomicBoolean +import javax.servlet.DispatcherType -import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future import scala.util.{Failure, Success} import scala.util.control.NonFatal -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} +import org.eclipse.jetty.servlet.{FilterHolder, FilterMapping} import org.apache.spark.SparkContext -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -170,10 +172,36 @@ private[spark] abstract class YarnSchedulerBackend( filterName != null && filterName.nonEmpty && filterParams != null && filterParams.nonEmpty if (hasFilter) { + // SPARK-26255: Append user provided filters(spark.ui.filters) with yarn filter. + val allFilters = Seq(filterName) ++ conf.get(UI_FILTERS) logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) - filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + + // For already installed handlers, prepend the filter. + scheduler.sc.ui.foreach { ui => + // Lock the UI so that new handlers are not added while this is running. Set the updated + // filter config inside the lock so that we're sure all handlers will properly get it. + ui.synchronized { + filterParams.foreach { case (k, v) => + conf.set(s"spark.$filterName.param.$k", v) + } + conf.set(UI_FILTERS, allFilters) + + ui.getHandlers.map(_.getServletHandler()).foreach { h => + val holder = new FilterHolder() + holder.setName(filterName) + holder.setClassName(filterName) + filterParams.foreach { case (k, v) => holder.setInitParameter(k, v) } + h.addFilter(holder) + + val mapping = new FilterMapping() + mapping.setFilterName(filterName) + mapping.setPathSpec("/*") + mapping.setDispatcherTypes(util.EnumSet.allOf(classOf[DispatcherType])) + + h.prependFilterMapping(mapping) + } + } + } } } @@ -270,7 +298,6 @@ private[spark] abstract class YarnSchedulerBackend( case u @ UpdateDelegationTokens(tokens) => // Add the tokens to the current user and send a message to the scheduler so that it // notifies all registered executors of the new tokens. - SparkHadoopUtil.get.addDelegationTokens(tokens, sc.conf) driverEndpoint.send(u) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 26013a109c42b..050008d672d13 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -23,6 +23,7 @@ import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap => MutableHashMap} +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -39,6 +40,7 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config._ import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { @@ -199,6 +201,20 @@ class ClientSuite extends SparkFunSuite with Matchers { appContext.getMaxAppAttempts should be (42) } + test("resource request (client mode)") { + val sparkConf = new SparkConf().set("spark.submit.deployMode", "client") + .set(YARN_AM_RESOURCE_TYPES_PREFIX + "fpga", "2") + .set(YARN_AM_RESOURCE_TYPES_PREFIX + "gpu", "3") + testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 2), ("gpu", 3))) + } + + test("resource request (cluster mode)") { + val sparkConf = new SparkConf().set("spark.submit.deployMode", "cluster") + .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "fpga", "4") + .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "gpu", "5") + testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 4), ("gpu", 5))) + } + test("spark.yarn.jars with multiple paths and globs") { val libs = Utils.createTempDir() val single = Utils.createTempDir() @@ -357,6 +373,35 @@ class ClientSuite extends SparkFunSuite with Matchers { sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName))) } + Seq( + "client" -> YARN_AM_RESOURCE_TYPES_PREFIX, + "cluster" -> YARN_DRIVER_RESOURCE_TYPES_PREFIX + ).foreach { case (deployMode, prefix) => + test(s"custom resource request ($deployMode mode)") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val resources = Map("fpga" -> 2, "gpu" -> 3) + ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) + + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, deployMode) + resources.foreach { case (name, v) => + conf.set(prefix + name, v.toString) + } + + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) + val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) + + val client = new Client(new ClientArguments(Array()), conf) + client.createApplicationSubmissionContext( + new YarnClientApplication(getNewApplicationResponse, appContext), + containerLaunchContext) + + resources.foreach { case (name, value) => + ResourceRequestTestHelper.getRequestedValue(appContext.getResource, name) should be (value) + } + } + } + private val matching = Seq( ("files URI match test1", "file:///file1", "file:///file2"), ("files URI match test2", "file:///c:file1", "file://c:file2"), @@ -433,4 +478,30 @@ class ClientSuite extends SparkFunSuite with Matchers { classpath(env) } + private def testResourceRequest( + sparkConf: SparkConf, + resources: List[String], + expectedResources: Seq[(String, Long)]): Unit = { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(resources) + + val args = new ClientArguments(Array()) + + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) + val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) + + val client = new Client(args, sparkConf) + client.createApplicationSubmissionContext( + new YarnClientApplication(getNewApplicationResponse, appContext), + containerLaunchContext) + + appContext.getAMContainerSpec should be (containerLaunchContext) + appContext.getApplicationType should be ("SPARK") + + expectedResources.foreach { case (name, value) => + ResourceRequestTestHelper.getRequestedValue(appContext.getResource, name) should be (value) + } + } + } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala new file mode 100644 index 0000000000000..535f45f411432 --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import org.apache.hadoop.yarn.api.records.Resource +import org.apache.hadoop.yarn.util.Records +import org.scalatest.Matchers + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation +import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config.{DRIVER_CORES, DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_MEMORY} + +class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { + + private val CUSTOM_RES_1 = "custom-resource-type-1" + private val CUSTOM_RES_2 = "custom-resource-type-2" + private val MEMORY = "memory" + private val CORES = "cores" + private val NEW_CONFIG_EXECUTOR_MEMORY = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + MEMORY + private val NEW_CONFIG_EXECUTOR_CORES = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + CORES + private val NEW_CONFIG_AM_MEMORY = YARN_AM_RESOURCE_TYPES_PREFIX + MEMORY + private val NEW_CONFIG_AM_CORES = YARN_AM_RESOURCE_TYPES_PREFIX + CORES + private val NEW_CONFIG_DRIVER_MEMORY = YARN_DRIVER_RESOURCE_TYPES_PREFIX + MEMORY + private val NEW_CONFIG_DRIVER_CORES = YARN_DRIVER_RESOURCE_TYPES_PREFIX + CORES + + test("resource request value does not match pattern") { + verifySetResourceRequestsException(List(CUSTOM_RES_1), + Map(CUSTOM_RES_1 -> "**@#"), CUSTOM_RES_1) + } + + test("resource request just unit defined") { + verifySetResourceRequestsException(List(), Map(CUSTOM_RES_1 -> "m"), CUSTOM_RES_1) + } + + test("resource request with null value should not be allowed") { + verifySetResourceRequestsException(List(), null, Map(CUSTOM_RES_1 -> "123"), + "requirement failed: Resource parameter should not be null!") + } + + test("resource request with valid value and invalid unit") { + verifySetResourceRequestsException(List(CUSTOM_RES_1), createResource, + Map(CUSTOM_RES_1 -> "123ppp"), "") + } + + test("resource request with valid value and without unit") { + verifySetResourceRequestsSuccessful(List(CUSTOM_RES_1), Map(CUSTOM_RES_1 -> "123"), + Map(CUSTOM_RES_1 -> ResourceInformation(CUSTOM_RES_1, 123, ""))) + } + + test("resource request with valid value and unit") { + verifySetResourceRequestsSuccessful(List(CUSTOM_RES_1), Map(CUSTOM_RES_1 -> "2g"), + Map(CUSTOM_RES_1 -> ResourceInformation(CUSTOM_RES_1, 2, "G"))) + } + + test("two resource requests with valid values and units") { + verifySetResourceRequestsSuccessful(List(CUSTOM_RES_1, CUSTOM_RES_2), + Map(CUSTOM_RES_1 -> "123m", CUSTOM_RES_2 -> "10G"), + Map(CUSTOM_RES_1 -> ResourceInformation(CUSTOM_RES_1, 123, "m"), + CUSTOM_RES_2 -> ResourceInformation(CUSTOM_RES_2, 10, "G"))) + } + + test("empty SparkConf should be valid") { + val sparkConf = new SparkConf() + ResourceRequestHelper.validateResources(sparkConf) + } + + test("just normal resources are defined") { + val sparkConf = new SparkConf() + sparkConf.set(DRIVER_MEMORY.key, "3G") + sparkConf.set(DRIVER_CORES.key, "4") + sparkConf.set(EXECUTOR_MEMORY.key, "4G") + sparkConf.set(EXECUTOR_CORES.key, "2") + ResourceRequestHelper.validateResources(sparkConf) + } + + Seq( + "value with unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 2, "G")), + "value without unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "")), + "multiple resources" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "m"), + ResourceInformation(CUSTOM_RES_2, 10, "G")) + ).foreach { case (name, resources) => + test(s"valid request: $name") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val resourceDefs = resources.map { r => r.name } + val requests = resources.map { r => (r.name, r.value.toString + r.unit) }.toMap + + ResourceRequestTestHelper.initializeResourceTypes(resourceDefs) + + val resource = createResource() + ResourceRequestHelper.setResourceRequests(requests, resource) + + resources.foreach { r => + val requested = ResourceRequestTestHelper.getResourceInformationByName(resource, r.name) + assert(requested === r) + } + } + } + + Seq( + ("value does not match pattern", CUSTOM_RES_1, "**@#"), + ("only unit defined", CUSTOM_RES_1, "m"), + ("invalid unit", CUSTOM_RES_1, "123ppp") + ).foreach { case (name, key, value) => + test(s"invalid request: $name") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(Seq(key)) + + val resource = createResource() + val thrown = intercept[IllegalArgumentException] { + ResourceRequestHelper.setResourceRequests(Map(key -> value), resource) + } + thrown.getMessage should include (key) + } + } + + Seq( + NEW_CONFIG_EXECUTOR_MEMORY -> "30G", + YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb" -> "30G", + YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb" -> "30G", + NEW_CONFIG_EXECUTOR_CORES -> "5", + YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores" -> "5", + NEW_CONFIG_AM_MEMORY -> "1G", + NEW_CONFIG_DRIVER_MEMORY -> "1G", + NEW_CONFIG_AM_CORES -> "3", + NEW_CONFIG_DRIVER_CORES -> "1G" + ).foreach { case (key, value) => + test(s"disallowed resource request: $key") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val conf = new SparkConf(false).set(key, value) + val thrown = intercept[SparkException] { + ResourceRequestHelper.validateResources(conf) + } + thrown.getMessage should include (key) + } + } + + test("multiple disallowed resources in config") { + val sparkConf = new SparkConf() + sparkConf.set(DRIVER_MEMORY.key, "2G") + sparkConf.set(DRIVER_CORES.key, "2") + sparkConf.set(EXECUTOR_MEMORY.key, "2G") + sparkConf.set(EXECUTOR_CORES.key, "4") + sparkConf.set(AM_MEMORY.key, "3G") + sparkConf.set(NEW_CONFIG_EXECUTOR_MEMORY, "3G") + sparkConf.set(NEW_CONFIG_AM_MEMORY, "2G") + sparkConf.set(NEW_CONFIG_DRIVER_MEMORY, "2G") + + val thrown = intercept[SparkException] { + ResourceRequestHelper.validateResources(sparkConf) + } + thrown.getMessage should ( + include(NEW_CONFIG_EXECUTOR_MEMORY) and + include(NEW_CONFIG_AM_MEMORY) and + include(NEW_CONFIG_DRIVER_MEMORY)) + } + + private def verifySetResourceRequestsSuccessful( + definedResourceTypes: List[String], + resourceRequests: Map[String, String], + expectedResources: Map[String, ResourceInformation]): Unit = { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(definedResourceTypes) + + val resource = createResource() + ResourceRequestHelper.setResourceRequests(resourceRequests, resource) + + expectedResources.foreach { case (name, ri) => + val resourceInfo = ResourceRequestTestHelper.getResourceInformationByName(resource, name) + assert(resourceInfo === ri) + } + } + + private def verifySetResourceRequestsException( + definedResourceTypes: List[String], + resourceRequests: Map[String, String], + message: String): Unit = { + val resource = createResource() + verifySetResourceRequestsException(definedResourceTypes, resource, resourceRequests, message) + } + + private def verifySetResourceRequestsException( + definedResourceTypes: List[String], + resource: Resource, + resourceRequests: Map[String, String], + message: String) = { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(definedResourceTypes) + val thrown = intercept[IllegalArgumentException] { + ResourceRequestHelper.setResourceRequests(resourceRequests, resource) + } + if (!message.isEmpty) { + thrown.getMessage should include (message) + } + } + + private def verifyValidateResourcesException(sparkConf: SparkConf, message: String) = { + val thrown = intercept[SparkException] { + ResourceRequestHelper.validateResources(sparkConf) + } + thrown.getMessage should include (message) + } + + private def createResource(): Resource = { + val resource = Records.newRecord(classOf[Resource]) + resource.setMemory(512) + resource.setVirtualCores(2) + resource + } +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala new file mode 100644 index 0000000000000..953d447bf4b6e --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.yarn.api.records.Resource + +import org.apache.spark.util.Utils + +object ResourceRequestTestHelper { + def initializeResourceTypes(resourceTypes: Seq[String]): Unit = { + if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { + throw new IllegalStateException("This method should not be invoked " + + "since YARN resource types is not available because of old Hadoop version!" ) + } + + // ResourceUtils.reinitializeResources() is the YARN-way + // to specify resources for the execution of the tests. + // This method should receive standard resources with names of memory-mb and vcores. + // Without specifying the standard resources or specifying them + // with different names e.g. memory, YARN would throw various exceptions + // because it relies on that standard resources are always specified. + val defaultResourceTypes = List( + createResourceTypeInfo("memory-mb"), + createResourceTypeInfo("vcores")) + val customResourceTypes = resourceTypes.map(createResourceTypeInfo) + val allResourceTypes = defaultResourceTypes ++ customResourceTypes + + val resourceUtilsClass = + Utils.classForName("org.apache.hadoop.yarn.util.resource.ResourceUtils") + val reinitializeResourcesMethod = resourceUtilsClass.getMethod("reinitializeResources", + classOf[java.util.List[AnyRef]]) + reinitializeResourcesMethod.invoke(null, allResourceTypes.asJava) + } + + private def createResourceTypeInfo(resourceName: String): AnyRef = { + val resTypeInfoClass = Utils.classForName("org.apache.hadoop.yarn.api.records.ResourceTypeInfo") + val resTypeInfoNewInstanceMethod = resTypeInfoClass.getMethod("newInstance", classOf[String]) + resTypeInfoNewInstanceMethod.invoke(null, resourceName) + } + + def getRequestedValue(res: Resource, rtype: String): AnyRef = { + val resourceInformation = getResourceInformation(res, rtype) + invokeMethod(resourceInformation, "getValue") + } + + def getResourceInformationByName(res: Resource, nameParam: String): ResourceInformation = { + val resourceInformation: AnyRef = getResourceInformation(res, nameParam) + val name = invokeMethod(resourceInformation, "getName").asInstanceOf[String] + val value = invokeMethod(resourceInformation, "getValue").asInstanceOf[Long] + val units = invokeMethod(resourceInformation, "getUnits").asInstanceOf[String] + ResourceInformation(name, value, units) + } + + private def getResourceInformation(res: Resource, name: String): AnyRef = { + if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { + throw new IllegalStateException("assertResourceTypeValue() should not be invoked " + + "since yarn resource types is not available because of old Hadoop version!") + } + + val getResourceInformationMethod = res.getClass.getMethod("getResourceInformation", + classOf[String]) + val resourceInformation = getResourceInformationMethod.invoke(res, name) + resourceInformation + } + + private def invokeMethod(resourceInformation: AnyRef, methodName: String): AnyRef = { + val getValueMethod = resourceInformation.getClass.getMethod(methodName) + getValueMethod.invoke(resourceInformation) + } + + case class ResourceInformation(name: String, value: Long, unit: String) +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 2fb892ecbc33a..ee848123f5923 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -124,7 +124,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val containerId: ContainerId = ContainerId.newContainerId(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) - Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null) + Container.newInstance(containerId, nodeId, "", resource, RM_REQUEST_PRIORITY, null) } def createContainers(hosts: Seq[String], containerIds: Seq[Int]): Seq[Container] = { @@ -158,6 +158,29 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter size should be (0) } + test("custom resource requested from yarn") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) + + val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) + val handler = createAllocator(1, mockAmClient, + Map(YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "gpu" -> "2G")) + + handler.updateResourceRequests() + val container = createContainer("host1", resource = handler.resource) + handler.handleAllocatedContainers(Array(container)) + + // get amount of memory and vcores from resource, so effectively skipping their validation + val expectedResources = Resource.newInstance(handler.resource.getMemory(), + handler.resource.getVirtualCores) + ResourceRequestHelper.setResourceRequests(Map("gpu" -> "2G"), expectedResources) + val captor = ArgumentCaptor.forClass(classOf[ContainerRequest]) + + verify(mockAmClient).addContainerRequest(captor.capture()) + val containerRequest: ContainerRequest = captor.getValue + assert(containerRequest.getCapability === expectedResources) + } + test("container should not be created if requested number if met") { // request a single container and receive it val handler = createAllocator(1) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 58d11e96942e1..5354bf7a7a693 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -36,6 +36,8 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher._ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} @@ -192,7 +194,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { val propsFile = createConfFile() val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setPropertiesFile(propsFile) .setMaster("yarn") .setDeployMode("client") @@ -441,7 +443,7 @@ private object YarnClusterDriver extends Logging with Matchers { // If we are running in yarn-cluster mode, verify that driver logs links and present and are // in the expected format. - if (conf.get("spark.submit.deployMode") == "cluster") { + if (conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { assert(listener.driverLogs.nonEmpty) val driverLogs = listener.driverLogs.get assert(driverLogs.size === 2) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 61c0c43f7c04f..de7ff8238c9e7 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -30,6 +30,8 @@ import org.scalatest.Matchers import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.{ResetSystemProperties, Utils} class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging @@ -83,7 +85,7 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // spark acls on, just pick up default user val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") + sparkConf.set(ACLS_ENABLE, true) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) @@ -111,9 +113,9 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // default spark acls are on and specify acls val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") - sparkConf.set("spark.ui.view.acls", "user1,user2") - sparkConf.set("spark.modify.acls", "user3,user4") + sparkConf.set(ACLS_ENABLE, true) + sparkConf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + sparkConf.set(MODIFY_ACLS, Seq("user3", "user4")) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala index 9fa749b14c98c..98315e4235741 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala @@ -19,11 +19,10 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.Credentials -import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { +class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite { private var credentialManager: YARNHadoopDelegationTokenManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -36,7 +35,7 @@ class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers test("Correctly loads credential providers") { credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - credentialManager.credentialProviders.get("yarn-test") should not be (None) + assert(credentialManager.isProviderLoaded("yarn-test")) } } diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ea20aef3531af..d13df8ebb4253 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -60,6 +60,10 @@ org.apache.spark spark-tags_${scala.binary.version} + + io.growing + bitmap +